diff --git a/.travis.yml b/.travis.yml index d36c0cb709983..532099c555b3a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,10 +20,12 @@ jdk: - openjdk8 jobs: include: - - name: "Unit tests except hudi-spark-client" - env: MODE=unit MODULES='!hudi-client/hudi-spark-client' HUDI_QUIETER_LOGGING=1 - name: "Unit tests for hudi-spark-client" env: MODE=unit MODULES=hudi-client/hudi-spark-client HUDI_QUIETER_LOGGING=1 + - name: "Unit tests for hudi-utilities" + env: MODE=unit MODULES=hudi-utilities HUDI_QUIETER_LOGGING=1 + - name: "All other unit tests" + env: MODE=unit MODULES='!hudi-utilities,!hudi-client/hudi-spark-client' HUDI_QUIETER_LOGGING=1 - name: "Functional tests" env: MODE=functional HUDI_QUIETER_LOGGING=1 - name: "Integration tests" diff --git a/LICENSE b/LICENSE index 1e2174731f161..385191d1b9efa 100644 --- a/LICENSE +++ b/LICENSE @@ -246,6 +246,8 @@ This product includes code from Apache Spark * org.apache.hudi.AvroConversionHelper copied from classes in org/apache/spark/sql/avro package +* org.apache.hudi.HoodieSparkUtils.scala copied some methods from org.apache.spark.deploy.SparkHadoopUtil.scala + Copyright: 2014 and onwards The Apache Software Foundation Home page: http://spark.apache.org/ License: http://www.apache.org/licenses/LICENSE-2.0 diff --git a/README.md b/README.md index 2416b3f92cc90..427d8595f4365 100644 --- a/README.md +++ b/README.md @@ -76,6 +76,14 @@ The default Scala version supported is 2.11. To build for Scala 2.12 version, bu mvn clean package -DskipTests -Dscala-2.12 ``` +### Build with Spark 3.0.0 + +The default Spark version supported is 2.4.4. To build for Spark 3.0.0 version, build using `spark3` profile + +``` +mvn clean package -DskipTests -Dspark3 +``` + ### Build without spark-avro module The default hudi-jar bundles spark-avro module. To build without spark-avro module, build using `spark-shade-unbundle-avro` profile diff --git a/docker/demo/config/test-suite/complex-dag-cow.yaml b/docker/demo/config/test-suite/complex-dag-cow.yaml index a10026c0b948b..5fa8596830f38 100644 --- a/docker/demo/config/test-suite/complex-dag-cow.yaml +++ b/docker/demo/config/test-suite/complex-dag-cow.yaml @@ -13,122 +13,56 @@ # 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. -first_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 1000 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 10000 - deps: first_insert - type: InsertNode -third_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 300 - deps: second_insert - type: InsertNode -first_rollback: - config: - deps: third_insert - type: RollbackNode -first_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_rollback -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 11300 - type: HiveQueryNode - deps: first_hive_sync -second_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_hive_query -second_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 11600 - type: HiveQueryNode - deps: second_upsert -fourth_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 1000 - deps: second_hive_query - type: InsertNode -third_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 12600 - type: HiveQueryNode - deps: fourth_insert -first_delete: - config: - record_size: 70000 - num_partitions_delete: 1 - num_records_delete: 200 - deps: third_hive_query - type: DeleteNode -fourth_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete -fourth_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 12400 - type: HiveQueryNode - deps: fourth_hive_sync \ No newline at end of file +dag_name: cow-long-running-example.yaml +dag_rounds: 2 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_validate: + config: + type: ValidateDatasetNode + deps: third_insert + first_upsert: + config: + record_size: 100 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 1 + num_records_delete: 2000 + type: DeleteNode + deps: first_upsert + second_validate: + config: + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/complex-dag-mor.yaml b/docker/demo/config/test-suite/complex-dag-mor.yaml index 2652b03070a7e..505e5e2945f9b 100644 --- a/docker/demo/config/test-suite/complex-dag-mor.yaml +++ b/docker/demo/config/test-suite/complex-dag-mor.yaml @@ -13,103 +13,107 @@ # 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. -first_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 100 - deps: first_insert - type: InsertNode -third_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 300 - deps: second_insert - type: InsertNode -first_rollback: - config: - deps: third_insert - type: RollbackNode -first_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_rollback -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveQueryNode - deps: first_hive_sync -second_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_hive_query -second_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 1100 - type: HiveQueryNode - deps: second_upsert -first_schedule_compact: - config: - type: ScheduleCompactNode - deps: second_hive_query -third_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_schedule_compact -first_compact: - config: - type: CompactNode - deps: first_schedule_compact -third_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 1400 - type: HiveQueryNode - deps: first_compact +dag_name: complex-dag-mor.yaml +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 5 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 5 + num_records_insert: 100 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_rollback: + config: + deps: third_insert + type: RollbackNode + first_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_rollback + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveQueryNode + deps: first_hive_sync + second_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_hive_query + second_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + hive_queries: + query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" + result1: 0 + query2: "select count(*) from testdb.table1" + result2: 1100 + type: HiveQueryNode + deps: second_upsert + first_schedule_compact: + config: + type: ScheduleCompactNode + deps: second_hive_query + third_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_schedule_compact + first_compact: + config: + type: CompactNode + deps: first_schedule_compact + third_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + hive_queries: + query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" + result1: 0 + query2: "select count(*) from testdb.table1" + result2: 1400 + type: HiveQueryNode + deps: first_compact diff --git a/docker/demo/config/test-suite/cow-long-running-example.yaml b/docker/demo/config/test-suite/cow-long-running-example.yaml new file mode 100644 index 0000000000000..b7026f2dd653a --- /dev/null +++ b/docker/demo/config/test-suite/cow-long-running-example.yaml @@ -0,0 +1,68 @@ +# 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. +dag_name: cow-long-running-example.yaml +dag_rounds: 20 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_validate: + config: + type: ValidateDatasetNode + deps: third_insert + first_upsert: + config: + record_size: 100 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 1 + num_records_delete: 2000 + type: DeleteNode + deps: first_upsert + second_validate: + config: + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/test-source.properties b/docker/demo/config/test-suite/test-source.properties deleted file mode 100644 index cc18a39d57086..0000000000000 --- a/docker/demo/config/test-suite/test-source.properties +++ /dev/null @@ -1,37 +0,0 @@ -# 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. - -# write configs -hoodie.datasource.write.recordkey.field=_row_key -hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator -hoodie.datasource.write.partitionpath.field=timestamp - - -# deltastreamer configs -hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd -hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP -hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-bench/input -hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/test-suite/source.avsc -hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc - -#hive sync -hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/ -hoodie.datasource.hive_sync.database=testdb -hoodie.datasource.hive_sync.table=table1 -hoodie.datasource.hive_sync.use_jdbc=false -hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor -hoodie.datasource.hive_sync.assume_date_partitioning=true -hoodie.datasource.hive_sync.use_pre_apache_input_format=true diff --git a/docker/demo/config/test-suite/test.properties b/docker/demo/config/test-suite/test.properties index a7fd3986a730b..0aa0f45c05ed5 100644 --- a/docker/demo/config/test-suite/test.properties +++ b/docker/demo/config/test-suite/test.properties @@ -1,3 +1,4 @@ + hoodie.insert.shuffle.parallelism=100 hoodie.upsert.shuffle.parallelism=100 hoodie.bulkinsert.shuffle.parallelism=100 @@ -8,6 +9,13 @@ hoodie.deltastreamer.source.test.max_unique_records=100000000 hoodie.embed.timeline.server=false hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.insert.shuffle.parallelism=100 +hoodie.upsert.shuffle.parallelism=100 +hoodie.bulkinsert.shuffle.parallelism=100 + +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.datasource.hive_sync.skip_ro_suffix=true + hoodie.datasource.write.recordkey.field=_row_key hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator hoodie.datasource.write.partitionpath.field=timestamp diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index febfcd28a1116..6409cc747a4be 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -21,6 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.spark.sql.SaveMode; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.HoodieDataSourceHelpers; +import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hadoop.fs.FileSystem; val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) @@ -49,6 +50,8 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive"). option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true"). option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr"). + option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor"); @@ -79,6 +82,8 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive"). option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true"). option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr"). + option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor_bs"); diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index 459379da4c14e..42eb15819bda7 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index f7406a18fbfb1..3ac8ec07326dc 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index da90fa07ecc40..b0c5a777d921b 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index 220483e7297e7..163fac2616e0f 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom @@ -59,7 +59,7 @@ - + diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index 6e1dfd23d72c4..dcd874ce020f2 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index efb1153a8b6a3..64b934239d13b 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../../pom.xml 4.0.0 diff --git a/docker/hoodie/hadoop/prestobase/pom.xml b/docker/hoodie/hadoop/prestobase/pom.xml index 5f3cd4cc00544..dea2f431170b0 100644 --- a/docker/hoodie/hadoop/prestobase/pom.xml +++ b/docker/hoodie/hadoop/prestobase/pom.xml @@ -16,13 +16,11 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index 98ad8c972a886..28d47d43f5359 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index 0ec0c1c9a4803..ffa9ab9ea3f39 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index 78758fc040e85..e7e546060493d 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index 486baad829988..89b42da21231b 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 pom diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index dda07f9a111dd..56ae066cb1eec 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index f8e82ae618581..e53dd38891604 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -193,13 +193,25 @@ public String showLogFileRecords( if (shouldMerge) { System.out.println("===========================> MERGING RECORDS <==================="); HoodieMergedLogRecordScanner scanner = - new HoodieMergedLogRecordScanner(fs, client.getBasePath(), logFilePaths, readerSchema, - client.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp(), - HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, - Boolean.parseBoolean(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), - Boolean.parseBoolean(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED), - HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE, - HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH); + HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(client.getBasePath()) + .withLogFilePaths(logFilePaths) + .withReaderSchema(readerSchema) + .withLatestInstantTime( + client.getActiveTimeline() + .getCommitTimeline().lastInstant().get().getTimestamp()) + .withReadBlocksLazily( + Boolean.parseBoolean( + HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)) + .withReverseReader( + Boolean.parseBoolean( + HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED)) + .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE) + .withMaxMemorySizeInBytes( + HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES) + .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH) + .build(); for (HoodieRecord hoodieRecord : scanner) { Option record = hoodieRecord.getData().getInsertValue(readerSchema); if (allRecords.size() < limit) { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java new file mode 100644 index 0000000000000..5b005540ab778 --- /dev/null +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -0,0 +1,231 @@ +/* + * 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.cli.commands; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.cli.HoodieCLI; +import org.apache.hudi.cli.utils.SparkUtil; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; + +import org.apache.spark.api.java.JavaSparkContext; +import org.springframework.shell.core.CommandMarker; +import org.springframework.shell.core.annotation.CliCommand; +import org.springframework.shell.core.annotation.CliOption; +import org.springframework.stereotype.Component; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * CLI commands to operate on the Metadata Table. + */ +@Component +public class MetadataCommand implements CommandMarker { + + private JavaSparkContext jsc; + private static String metadataBaseDirectory; + + /** + * Sets the directory to store/read Metadata Table. + * + * This can be used to store the metadata table away from the dataset directory. + * - Useful for testing as well as for using via the HUDI CLI so that the actual dataset is not written to. + * - Useful for testing Metadata Table performance and operations on existing datasets before enabling. + */ + public static void setMetadataBaseDirectory(String metadataDir) { + ValidationUtils.checkState(metadataBaseDirectory == null, + "metadataBaseDirectory is already set to " + metadataBaseDirectory); + metadataBaseDirectory = metadataDir; + } + + public static String getMetadataTableBasePath(String tableBasePath) { + if (metadataBaseDirectory != null) { + return metadataBaseDirectory; + } + return HoodieTableMetadata.getMetadataTableBasePath(tableBasePath); + } + + @CliCommand(value = "metadata set", help = "Set options for Metadata Table") + public String set(@CliOption(key = {"metadataDir"}, + help = "Directory to read/write metadata table (can be different from dataset)", unspecifiedDefaultValue = "") + final String metadataDir) { + if (!metadataDir.isEmpty()) { + setMetadataBaseDirectory(metadataDir); + } + + return "Ok"; + } + + @CliCommand(value = "metadata create", help = "Create the Metadata Table if it does not exist") + public String create() throws IOException { + HoodieCLI.getTableMetaClient(); + Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); + try { + FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath); + if (statuses.length > 0) { + throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") not empty."); + } + } catch (FileNotFoundException e) { + // Metadata directory does not exist yet + HoodieCLI.fs.mkdirs(metadataPath); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieWriteConfig writeConfig = getWriteConfig(); + initJavaSparkContext(); + SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); + return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0); + } + + @CliCommand(value = "metadata delete", help = "Remove the Metadata Table") + public String delete() throws Exception { + HoodieCLI.getTableMetaClient(); + Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); + try { + FileStatus[] statuses = HoodieCLI.fs.listStatus(metadataPath); + if (statuses.length > 0) { + HoodieCLI.fs.delete(metadataPath, true); + } + } catch (FileNotFoundException e) { + // Metadata directory does not exist + } + + return String.format("Removed Metadata Table from %s", metadataPath); + } + + @CliCommand(value = "metadata init", help = "Update the metadata table from commits since the creation") + public String init(@CliOption(key = {"readonly"}, unspecifiedDefaultValue = "false", + help = "Open in read-only mode") final boolean readOnly) throws Exception { + HoodieCLI.getTableMetaClient(); + Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); + try { + HoodieCLI.fs.listStatus(metadataPath); + } catch (FileNotFoundException e) { + // Metadata directory does not exist + throw new RuntimeException("Metadata directory (" + metadataPath.toString() + ") does not exist."); + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + if (!readOnly) { + HoodieWriteConfig writeConfig = getWriteConfig(); + initJavaSparkContext(); + SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); + } + + String action = readOnly ? "Opened" : "Initialized"; + return String.format(action + " Metadata Table in %s (duration=%.2fsec)", metadataPath, (timer.endTimer()) / 1000.0); + } + + @CliCommand(value = "metadata stats", help = "Print stats about the metadata") + public String stats() throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); + HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(HoodieCLI.conf, config, HoodieCLI.basePath, "/tmp"); + Map stats = metadata.stats(); + + StringBuffer out = new StringBuffer("\n"); + out.append(String.format("Base path: %s\n", getMetadataTableBasePath(HoodieCLI.basePath))); + for (Map.Entry entry : stats.entrySet()) { + out.append(String.format("%s: %s\n", entry.getKey(), entry.getValue())); + } + + return out.toString(); + } + + @CliCommand(value = "metadata list-partitions", help = "Print a list of all partitions from the metadata") + public String listPartitions() throws IOException { + HoodieCLI.getTableMetaClient(); + initJavaSparkContext(); + HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); + HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, + HoodieCLI.basePath, "/tmp"); + + StringBuffer out = new StringBuffer("\n"); + if (!metadata.enabled()) { + out.append("=== Metadata Table not initilized. Using file listing to get list of partitions. ===\n\n"); + } + + long t1 = System.currentTimeMillis(); + List partitions = metadata.getAllPartitionPaths(); + long t2 = System.currentTimeMillis(); + + int[] count = {0}; + partitions.stream().sorted((p1, p2) -> p2.compareTo(p1)).forEach(p -> { + out.append(p); + if (++count[0] % 15 == 0) { + out.append("\n"); + } else { + out.append(", "); + } + }); + + out.append(String.format("\n\n=== List of partitions retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + + return out.toString(); + } + + @CliCommand(value = "metadata list-files", help = "Print a list of all files in a partition from the metadata") + public String listFiles( + @CliOption(key = {"partition"}, help = "Name of the partition to list files", mandatory = true) + final String partition) throws IOException { + HoodieCLI.getTableMetaClient(); + HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); + HoodieBackedTableMetadata metaReader = new HoodieBackedTableMetadata(HoodieCLI.conf, config, HoodieCLI.basePath, "/tmp"); + + StringBuffer out = new StringBuffer("\n"); + if (!metaReader.enabled()) { + out.append("=== Metadata Table not initialized. Using file listing to get list of files in partition. ===\n\n"); + } + + long t1 = System.currentTimeMillis(); + FileStatus[] statuses = metaReader.getAllFilesInPartition(new Path(HoodieCLI.basePath, partition)); + long t2 = System.currentTimeMillis(); + + Arrays.stream(statuses).sorted((p1, p2) -> p2.getPath().getName().compareTo(p1.getPath().getName())).forEach(p -> { + out.append("\t" + p.getPath().getName()); + out.append("\n"); + }); + + out.append(String.format("\n=== Files in partition retrieved in %.2fsec ===", (t2 - t1) / 1000.0)); + + return out.toString(); + } + + private HoodieWriteConfig getWriteConfig() { + return HoodieWriteConfig.newBuilder().withPath(HoodieCLI.basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); + } + + private void initJavaSparkContext() { + if (jsc == null) { + jsc = SparkUtil.initJavaSparkConf("HoodieClI"); + } + } +} diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java index 0c52220e03155..78460b1763c69 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java @@ -104,7 +104,7 @@ public void init() throws IOException, InterruptedException, URISyntaxException header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = new HoodieAvroDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); } finally { if (writer != null) { writer.close(); @@ -183,7 +183,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, INSTANT_TIME); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); } finally { if (writer != null) { writer.close(); @@ -197,13 +197,23 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc // get expected result of 10 records. List logFilePaths = Arrays.stream(fs.globStatus(new Path(partitionPath + "/*"))) .map(status -> status.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = - new HoodieMergedLogRecordScanner(fs, tablePath, logFilePaths, schema, INSTANT_TIME, - HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, - Boolean.parseBoolean(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED), - Boolean.parseBoolean(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED), - HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE, - HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(tablePath) + .withLogFilePaths(logFilePaths) + .withReaderSchema(schema) + .withLatestInstantTime(INSTANT_TIME) + .withMaxMemorySizeInBytes( + HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES) + .withReadBlocksLazily( + Boolean.parseBoolean( + HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)) + .withReverseReader( + Boolean.parseBoolean( + HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED)) + .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE) + .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH) + .build(); Iterator> records = scanner.iterator(); int num = 0; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java index 133dcb0577bab..9572af5237d25 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java @@ -31,7 +31,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.SchemaTestUtil; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileStatus; @@ -80,7 +80,7 @@ public void init() throws Exception { // generate 200 records Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); HoodieRecord[] hoodieRecords1 = SchemaTestUtil.generateHoodieTestRecords(0, 100, schema).toArray(new HoodieRecord[100]); HoodieRecord[] hoodieRecords2 = SchemaTestUtil.generateHoodieTestRecords(100, 100, schema).toArray(new HoodieRecord[100]); diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 487a2e2b3ea8f..977765f4bd03f 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -15,29 +15,21 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi-client org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 hudi-client-common - ${parent.version} + 0.8.0-SNAPSHOT hudi-client-common jar - - - org.scala-lang - scala-library - ${scala.version} - - org.apache.hudi diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java index 47f883284adb8..1bb5daa528c79 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java @@ -19,8 +19,8 @@ import org.apache.hudi.client.AbstractCompactor; import org.apache.hudi.client.AbstractHoodieWriteClient; -import org.apache.hudi.client.common.EngineProperty; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -53,7 +53,7 @@ public abstract class AsyncCompactService extends HoodieAsyncService { private final int maxConcurrentCompaction; private transient AbstractCompactor compactor; - private transient HoodieEngineContext context; + protected transient HoodieEngineContext context; private transient BlockingQueue pendingCompactions = new LinkedBlockingQueue<>(); private transient ReentrantLock queueLock = new ReentrantLock(); private transient Condition consumed = queueLock.newCondition(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index e502281322fe7..765965fa56747 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -20,7 +20,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineServerHelper; import org.apache.hudi.client.embedded.EmbeddedTimelineService; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; @@ -71,6 +71,7 @@ protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig cl this.timelineServer = timelineServer; shouldStopTimelineServer = !timelineServer.isPresent(); startEmbeddedServerView(); + initWrapperFSMetrics(); } /** @@ -79,6 +80,7 @@ protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig cl @Override public void close() { stopEmbeddedServerView(true); + this.context.setJobStatus("", ""); } private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) { @@ -117,6 +119,14 @@ public HoodieWriteConfig getConfig() { return config; } + public HoodieEngineContext getEngineContext() { + return context; + } + + protected void initWrapperFSMetrics() { + // no-op. + } + protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad, config.getConsistencyGuardConfig(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 222e1ab2ca5b2..63d0bffbe467f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -21,6 +21,7 @@ import com.codahale.metrics.Timer; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -28,7 +29,7 @@ import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage; import org.apache.hudi.callback.util.HoodieCommitCallbackFactory; import org.apache.hudi.client.embedded.EmbeddedTimelineService; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -38,9 +39,12 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -88,11 +92,12 @@ public abstract class AbstractHoodieWriteClient table, HoodieCommitMetadata me } else { metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); } + + // Do an inline clustering if enabled + if (config.isInlineClustering()) { + runAnyPendingClustering(table); + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); + inlineCluster(extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); + } + // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); - autoCleanOnCommit(instantTime); + autoCleanOnCommit(); + + syncTableMetadata(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } @@ -406,12 +440,21 @@ protected void runAnyPendingCompactions(HoodieTable table) { }); } + protected void runAnyPendingClustering(HoodieTable table) { + table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> { + Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); + if (instantPlan.isPresent()) { + LOG.info("Running pending clustering at instant " + instantPlan.get().getLeft()); + cluster(instant.getTimestamp(), true); + } + }); + } + /** * Handle auto clean during commit. * - * @param instantTime */ - protected void autoCleanOnCommit(String instantTime) { + protected void autoCleanOnCommit() { if (config.isAutoClean()) { // Call clean to cleanup if there is anything to cleanup after the commit, if (config.isAsyncClean()) { @@ -419,8 +462,9 @@ protected void autoCleanOnCommit(String instantTime) { AsyncCleanerService.waitForCompletion(asyncCleanerService); LOG.info("Cleaner has finished"); } else { + // Do not reuse instantTime for clean as metadata table requires all changes to have unique instant timestamps. LOG.info("Auto cleaning is enabled. Running cleaner now"); - clean(instantTime); + clean(); } } } @@ -580,7 +624,8 @@ public String startCommit() { rollbackPendingCommits(); } String instantTime = HoodieActiveTimeline.createNewInstantTime(); - startCommitWithTime(instantTime); + HoodieTableMetaClient metaClient = createMetaClient(true); + startCommit(instantTime, metaClient.getCommitActionType(), metaClient); return instantTime; } @@ -674,8 +719,7 @@ public abstract void commitCompaction(String compactionInstantTime, O writeStatu */ protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses, HoodieTable table, String compactionCommitTime); - - + /** * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file * @@ -687,12 +731,30 @@ public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTabl table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant); } + /** + * Get inflight time line exclude compaction and clustering. + * @param table + * @return + */ + private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTable table) { + HoodieTimeline inflightTimelineWithReplaceCommit = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimelineExcludeClusteringCommit = inflightTimelineWithReplaceCommit.filter(instant -> { + if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { + Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); + return !instantPlan.isPresent(); + } else { + return true; + } + }); + return inflightTimelineExcludeClusteringCommit; + } + /** * Cleanup all pending commits. */ private void rollbackPendingCommits() { HoodieTable table = createTable(config, hadoopConf); - HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); + HoodieTimeline inflightTimeline = getInflightTimelineExcludeCompactionAndClustering(table); List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) .collect(Collectors.toList()); for (String commit : commits) { @@ -726,6 +788,54 @@ protected Option inlineCompact(Option> extraMetadata return compactionInstantTimeOpt; } + /** + * Schedules a new clustering instant. + * + * @param extraMetadata Extra Metadata to be stored + */ + public Option scheduleClustering(Option> extraMetadata) throws HoodieIOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + } + + /** + * Schedules a new clustering instant with passed-in instant time. + * + * @param instantTime clustering Instant Time + * @param extraMetadata Extra Metadata to be stored + */ + public boolean scheduleClusteringAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { + LOG.info("Scheduling clustering at instant time :" + instantTime); + Option plan = createTable(config, hadoopConf) + .scheduleClustering(context, instantTime, extraMetadata); + return plan.isPresent(); + } + + /** + * Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata. + * + * @param clusteringInstant Clustering Instant Time + * @return Collection of Write Status + */ + public abstract HoodieWriteMetadata cluster(String clusteringInstant, boolean shouldComplete); + + /** + * Executes a clustering plan on a table, serially before or after an insert/upsert action. + */ + protected Option inlineCluster(Option> extraMetadata) { + Option clusteringInstantOpt = scheduleClustering(extraMetadata); + clusteringInstantOpt.ifPresent(clusteringInstant -> { + // inline cluster should auto commit as the user is never given control + cluster(clusteringInstant, true); + }); + return clusteringInstantOpt; + } + + protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false); + table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant); + } + /** * Finalize Write operation. * @@ -773,7 +883,8 @@ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { try { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); Option lastInstant = - activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType())) + activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType()) + || s.getAction().equals(HoodieActiveTimeline.REPLACE_COMMIT_ACTION)) .lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java index e8016c957be65..c259acf2df67f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java @@ -19,6 +19,7 @@ package org.apache.hudi.client; import org.apache.hudi.async.HoodieAsyncService; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; @@ -52,11 +53,11 @@ protected Pair startService() { }), executor); } - public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient, - String instantTime) { + public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient) { AsyncCleanerService asyncCleanerService = null; if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) { - LOG.info("Auto cleaning is enabled. Running cleaner async to write operation"); + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + LOG.info("Auto cleaning is enabled. Running cleaner async to write operation at instant time " + instantTime); asyncCleanerService = new AsyncCleanerService(writeClient, instantTime); asyncCleanerService.start(null); } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java index a2ecb67277afe..9ace03a6d50ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java index 515f43e64f108..ee03bd7cbf841 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; @@ -29,7 +30,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.TableFileSystemView; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -68,7 +68,7 @@ public static org.apache.hudi.avro.model.HoodieReplaceCommitMetadata convertRepl public static boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieTableMetaClient metaClient, TableFileSystemView fileSystemView, HoodieInstant instant, List replacedPartitions) { - + context.setJobStatus(ReplaceArchivalHelper.class.getSimpleName(), "Delete replaced file groups"); List f = context.map(replacedPartitions, partition -> { Stream fileSlices = fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition) .flatMap(HoodieFileGroup::getAllRawFileSlices); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java index 0a07ee5aa9f36..1cf1702717295 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.collection.Pair; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java index f63345d915da5..c8e552b1a8dd6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java index 1d5984794b1bc..fa74aa34eda66 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineServerHelper.java @@ -18,8 +18,8 @@ package org.apache.hudi.client.embedded; -import org.apache.hudi.client.common.EngineProperty; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -49,8 +49,8 @@ public static Option createEmbeddedTimelineService( // Run Embedded Timeline Server LOG.info("Starting Timeline service !!"); Option hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST); - timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null), - config.getEmbeddedTimelineServerPort(), config.getClientSpecifiedViewStorageConfig())); + timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null), config.getEmbeddedTimelineServerPort(), + config.getMetadataConfig(), config.getClientSpecifiedViewStorageConfig(), config.getBasePath())); timelineServer.get().startServer(); updateWriteConfigWithTimelineServer(timelineServer.get(), config); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index 738119cef2854..386f7d543dea8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -18,7 +18,8 @@ package org.apache.hudi.client.embedded; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -41,14 +42,22 @@ public class EmbeddedTimelineService { private int serverPort; private int preferredPort; private String hostAddr; + private HoodieEngineContext context; private final SerializableConfiguration hadoopConf; private final FileSystemViewStorageConfig config; + private final HoodieMetadataConfig metadataConfig; + private final String basePath; + private transient FileSystemViewManager viewManager; private transient TimelineService server; - public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, int embeddedTimelineServerPort, FileSystemViewStorageConfig config) { + public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, int embeddedTimelineServerPort, + HoodieMetadataConfig metadataConfig, FileSystemViewStorageConfig config, String basePath) { setHostAddr(embeddedTimelineServiceHostAddr); + this.context = context; this.config = config; + this.basePath = basePath; + this.metadataConfig = metadataConfig; this.hadoopConf = context.getHadoopConf(); this.viewManager = createViewManager(); this.preferredPort = embeddedTimelineServerPort; @@ -64,7 +73,7 @@ private FileSystemViewManager createViewManager() { // Reset to default if set to Remote builder.withStorageType(FileSystemViewStorageType.MEMORY); } - return FileSystemViewManager.createViewManager(hadoopConf, builder.build()); + return FileSystemViewManager.createViewManager(context, metadataConfig, builder.build(), basePath); } public void startServer() throws IOException { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java new file mode 100644 index 0000000000000..aa6c29b0844a2 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java @@ -0,0 +1,60 @@ +/* + * 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 org.apache.hudi.common.util.ValidationUtils; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +/** + * Provides iterator interface over List of iterators. Consumes all records from first iterator element + * before moving to next iterator in the list. That is concatenate elements across multiple iterators. + * + * @param + */ +public class ConcatenatingIterator implements Iterator { + + private final Queue> allIterators; + + public ConcatenatingIterator(List> iterators) { + allIterators = new LinkedList<>(iterators); + } + + @Override + public boolean hasNext() { + while (!allIterators.isEmpty()) { + if (allIterators.peek().hasNext()) { + return true; + } + // iterator at current head is done. move ahead + allIterators.poll(); + } + + return false; + } + + @Override + public T next() { + ValidationUtils.checkArgument(hasNext(), "No more elements left"); + return allIterators.peek().next(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java new file mode 100644 index 0000000000000..347f86cfba349 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; + +import java.util.List; +import java.util.Map; + +/** + * A utility class for calculating metrics related to FileSlice. + */ +public class FileSliceMetricUtils { + + public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB"; + public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB"; + public static final String TOTAL_IO_MB = "TOTAL_IO_MB"; + public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE"; + public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES"; + + public static void addFileSliceCommonMetrics(List fileSlices, Map metrics, long defaultBaseFileSize) { + int numLogFiles = 0; + long totalLogFileSize = 0; + long totalIORead = 0; + long totalIOWrite = 0; + long totalIO = 0; + + for (FileSlice slice : fileSlices) { + numLogFiles += slice.getLogFiles().count(); + // Total size of all the log files + totalLogFileSize += slice.getLogFiles().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) + .reduce(Long::sum).orElse(0L); + + long baseFileSize = slice.getBaseFile().isPresent() ? slice.getBaseFile().get().getFileSize() : 0L; + // Total read will be the base file + all the log files + totalIORead = FSUtils.getSizeInMB(baseFileSize + totalLogFileSize); + // Total write will be similar to the size of the base file + totalIOWrite = FSUtils.getSizeInMB(baseFileSize > 0 ? baseFileSize : defaultBaseFileSize); + // Total IO will the the IO for read + write + totalIO = totalIORead + totalIOWrite; + } + + metrics.put(TOTAL_IO_READ_MB, (double) totalIORead); + metrics.put(TOTAL_IO_WRITE_MB, (double) totalIOWrite); + metrics.put(TOTAL_IO_MB, (double) totalIO); + metrics.put(TOTAL_LOG_FILE_SIZE, (double) totalLogFileSize); + metrics.put(TOTAL_LOG_FILES, (double) numLogFiles); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java new file mode 100644 index 0000000000000..91acd3075377a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -0,0 +1,199 @@ +/* + * 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.common.config.DefaultHoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Clustering specific configs. + */ +public class HoodieClusteringConfig extends DefaultHoodieConfig { + + // Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy + public static final String CLUSTERING_PLAN_STRATEGY_CLASS = "hoodie.clustering.plan.strategy.class"; + public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS = + "org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy"; + + // Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy + public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = "hoodie.clustering.execution.strategy.class"; + public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS = + "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy"; + + // Turn on inline clustering - clustering will be run after write operation is complete. + public static final String INLINE_CLUSTERING_PROP = "hoodie.clustering.inline"; + private static final String DEFAULT_INLINE_CLUSTERING = "false"; + + // Config to control frequency of clustering + public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = "hoodie.clustering.inline.max.commits"; + private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = "4"; + + // Any strategy specific params can be saved with this prefix + public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; + + // Number of partitions to list to create ClusteringPlan. + public static final String CLUSTERING_TARGET_PARTITIONS = CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions"; + public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = String.valueOf(2); + + // Files smaller than the size specified here are candidates for clustering. + public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit"; + public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = String.valueOf(600 * 1024 * 1024L); // 600MB + + // Each clustering operation can create multiple groups. Total amount of data processed by clustering operation + // is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS). + // Max amount of data to be included in one group + public static final String CLUSTERING_MAX_BYTES_PER_GROUP = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group"; + public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = String.valueOf(2 * 1024 * 1024 * 1024L); + + // Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. + public static final String CLUSTERING_MAX_NUM_GROUPS = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups"; + public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = "30"; + + // Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups. + public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes"; + public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = String.valueOf(1 * 1024 * 1024 * 1024L); // 1GB + + // Constants related to clustering that may be used by more than 1 strategy. + public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns"; + + // When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update + public static final String CLUSTERING_UPDATES_STRATEGY_PROP = "hoodie.clustering.updates.strategy"; + public static final String DEFAULT_CLUSTERING_UPDATES_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy"; + + // Async clustering + public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = "hoodie.clustering.async.enabled"; + public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = "false"; + + public HoodieClusteringConfig(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 withClusteringPlanStrategyClass(String clusteringStrategyClass) { + props.setProperty(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass); + return this; + } + + public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) { + props.setProperty(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass); + return this; + } + + public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) { + props.setProperty(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions)); + return this; + } + + public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) { + props.setProperty(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit)); + return this; + } + + public Builder withClusteringSortColumns(String sortColumns) { + props.setProperty(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns); + return this; + } + + public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) { + props.setProperty(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize)); + return this; + } + + public Builder withClusteringMaxNumGroups(int maxNumGroups) { + props.setProperty(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups)); + return this; + } + + public Builder withClusteringTargetFileMaxBytes(long targetFileSize) { + props.setProperty(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize)); + return this; + } + + public Builder withInlineClustering(Boolean inlineClustering) { + props.setProperty(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering)); + return this; + } + + public Builder withInlineClusteringNumCommits(int numCommits) { + props.setProperty(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits)); + return this; + } + + public Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder withClusteringUpdatesStrategy(String updatesStrategyClass) { + props.setProperty(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass); + return this; + } + + public Builder withAsyncClustering(Boolean asyncClustering) { + props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering)); + return this; + } + + public HoodieClusteringConfig build() { + HoodieClusteringConfig config = new HoodieClusteringConfig(props); + + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_STRATEGY_CLASS), + CLUSTERING_PLAN_STRATEGY_CLASS, DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_EXECUTION_STRATEGY_CLASS), + CLUSTERING_EXECUTION_STRATEGY_CLASS, DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_BYTES_PER_GROUP), CLUSTERING_MAX_BYTES_PER_GROUP, + DEFAULT_CLUSTERING_MAX_GROUP_SIZE); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_NUM_GROUPS), CLUSTERING_MAX_NUM_GROUPS, + DEFAULT_CLUSTERING_MAX_NUM_GROUPS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_FILE_MAX_BYTES), CLUSTERING_TARGET_FILE_MAX_BYTES, + DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES); + setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_PROP), INLINE_CLUSTERING_PROP, + DEFAULT_INLINE_CLUSTERING); + setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_MAX_COMMIT_PROP), INLINE_CLUSTERING_MAX_COMMIT_PROP, + DEFAULT_INLINE_CLUSTERING_NUM_COMMITS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_PARTITIONS), CLUSTERING_TARGET_PARTITIONS, + DEFAULT_CLUSTERING_TARGET_PARTITIONS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_SMALL_FILE_LIMIT), CLUSTERING_PLAN_SMALL_FILE_LIMIT, + DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_UPDATES_STRATEGY_PROP), CLUSTERING_UPDATES_STRATEGY_PROP, + DEFAULT_CLUSTERING_UPDATES_STRATEGY); + setDefaultOnCondition(props, !props.containsKey(ASYNC_CLUSTERING_ENABLE_OPT_KEY), ASYNC_CLUSTERING_ENABLE_OPT_KEY, + DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL); + return config; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 83e9f678bc7ba..2fbd71dc6ab26 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -20,6 +20,8 @@ import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.config.DefaultHoodieConfig; +import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; import javax.annotation.concurrent.Immutable; @@ -36,7 +38,6 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public static final String INDEX_TYPE_PROP = "hoodie.index.type"; - public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name(); public static final String INDEX_CLASS_PROP = "hoodie.index.class"; public static final String DEFAULT_INDEX_CLASS = ""; @@ -103,8 +104,18 @@ public class HoodieIndexConfig extends DefaultHoodieConfig { public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = "hoodie.simple.index.update.partition.path"; public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = "false"; + private EngineType engineType; + + /** + * Use Spark engine by default. + */ private HoodieIndexConfig(Properties props) { + this(EngineType.SPARK, props); + } + + private HoodieIndexConfig(EngineType engineType, Properties props) { super(props); + this.engineType = engineType; } public static HoodieIndexConfig.Builder newBuilder() { @@ -113,6 +124,7 @@ public static HoodieIndexConfig.Builder newBuilder() { public static class Builder { + private EngineType engineType = EngineType.SPARK; private final Properties props = new Properties(); public Builder fromFile(File propertiesFile) throws IOException { @@ -237,9 +249,14 @@ public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionP return this; } + public Builder withEngineType(EngineType engineType) { + this.engineType = engineType; + return this; + } + public HoodieIndexConfig build() { - HoodieIndexConfig config = new HoodieIndexConfig(props); - setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE); + HoodieIndexConfig config = new HoodieIndexConfig(engineType, props); + setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, getDefaultIndexType(engineType)); setDefaultOnCondition(props, !props.containsKey(INDEX_CLASS_PROP), INDEX_CLASS_PROP, DEFAULT_INDEX_CLASS); setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES); @@ -260,9 +277,9 @@ public HoodieIndexConfig build() { BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING); setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP), BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET); - setDefaultOnCondition(props, !props.contains(BLOOM_INDEX_FILTER_TYPE), + setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_FILTER_TYPE), BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE); - setDefaultOnCondition(props, !props.contains(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES), + setDefaultOnCondition(props, !props.containsKey(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES), HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES); setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_PARALLELISM_PROP), SIMPLE_INDEX_PARALLELISM_PROP, DEFAULT_SIMPLE_INDEX_PARALLELISM); @@ -278,5 +295,20 @@ public HoodieIndexConfig build() { HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP)); return config; } + + private String getDefaultIndexType(EngineType engineType) { + switch (engineType) { + case SPARK: + return HoodieIndex.IndexType.BLOOM.name(); + case FLINK: + return HoodieIndex.IndexType.INMEMORY.name(); + default: + throw new HoodieNotSupportedException("Unsupported engine " + engineType); + } + } + + public EngineType getEngineType() { + return engineType; + } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java index 800c75f824fd4..b6cb6e5add99d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java @@ -62,6 +62,9 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig { public static final String METRICS_REPORTER_CLASS = METRIC_PREFIX + ".reporter.class"; public static final String DEFAULT_METRICS_REPORTER_CLASS = ""; + // Enable metrics collection from executors + public static final String ENABLE_EXECUTOR_METRICS = METRIC_PREFIX + ".executor.enable"; + private HoodieMetricsConfig(Properties props) { super(props); } @@ -126,6 +129,11 @@ public Builder withReporterClass(String className) { return this; } + public Builder withExecutorMetrics(boolean enable) { + props.setProperty(ENABLE_EXECUTOR_METRICS, String.valueOf(enable)); + return this; + } + public HoodieMetricsConfig build() { HoodieMetricsConfig config = new HoodieMetricsConfig(props); setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java new file mode 100644 index 0000000000000..442bd02c67a25 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java @@ -0,0 +1,74 @@ +/* + * 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.common.config.DefaultHoodieConfig; +import org.apache.hudi.config.HoodieMemoryConfig.Builder; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_ORDERING_FIELD_VAL; +import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP; + +/** + * Hoodie payload related configs. + */ +public class HoodiePayloadConfig extends DefaultHoodieConfig { + + public HoodiePayloadConfig(Properties props) { + super(props); + } + + public static HoodiePayloadConfig.Builder newBuilder() { + return new HoodiePayloadConfig.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 fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder withPayloadOrderingField(String payloadOrderingField) { + props.setProperty(PAYLOAD_ORDERING_FIELD_PROP, String.valueOf(payloadOrderingField)); + return this; + } + + public HoodiePayloadConfig build() { + HoodiePayloadConfig config = new HoodiePayloadConfig(props); + setDefaultOnCondition(props, !props.containsKey(PAYLOAD_ORDERING_FIELD_PROP), DEFAULT_PAYLOAD_ORDERING_FIELD_VAL, + String.valueOf(DEFAULT_PAYLOAD_ORDERING_FIELD_VAL)); + return config; + } + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 42d3e2b404568..d8135d44135b0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -18,21 +18,25 @@ package org.apache.hudi.config; -import org.apache.hadoop.hbase.io.compress.Compression; 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.HoodieMetadataConfig; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; +import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import javax.annotation.concurrent.Immutable; @@ -49,6 +53,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; + /** * Class storing configs for the HoodieWriteClient. */ @@ -58,6 +63,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final long serialVersionUID = 0L; public static final String TABLE_NAME = "hoodie.table.name"; + public static final String PRECOMBINE_FIELD_PROP = "hoodie.datasource.write.precombine.field"; + public static final String WRITE_PAYLOAD_CLASS = "hoodie.datasource.write.payload.class"; + public static final String DEFAULT_WRITE_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName(); + public static final String KEYGENERATOR_CLASS_PROP = "hoodie.datasource.write.keygenerator.class"; + public static final String DEFAULT_KEYGENERATOR_CLASS = SimpleAvroKeyGenerator.class.getName(); public static final String DEFAULT_ROLLBACK_USING_MARKERS = "false"; public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers"; public static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version"; @@ -69,6 +79,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; public static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; public static final String BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = "hoodie.bulkinsert.user.defined.partitioner.class"; + public static final String BULKINSERT_INPUT_DATA_SCHEMA_DDL = "hoodie.bulkinsert.schema.ddl"; public static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; public static final String DELETE_PARALLELISM = "hoodie.delete.shuffle.parallelism"; public static final String DEFAULT_ROLLBACK_PARALLELISM = "100"; @@ -85,8 +96,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER"; public static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit"; public static final String DEFAULT_HOODIE_AUTO_COMMIT = "true"; - public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning"; - public static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; + public static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class"; public static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName(); public static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism"; @@ -117,13 +127,17 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks"; public static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7; + // Data validation check performed during merges before actual commits + private static final String MERGE_DATA_VALIDATION_CHECK_ENABLED = "hoodie.merge.data.validation.enabled"; + private static final String DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED = "false"; + /** * HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow * multiple write operations (upsert/buk-insert/...) to be executed within a single commit. - * + *

* Given Hudi commit protocol, these are generally unsafe operations and user need to handle failure scenarios. It * only works with COW table. Hudi 0.5.x had stopped this behavior. - * + *

* Given the importance of supporting such cases for the user's migration to 0.5.x, we are proposing a safety flag * (disabled by default) which will allow this old behavior. */ @@ -140,14 +154,28 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { // We keep track of original config and rewritten config private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig; private FileSystemViewStorageConfig viewStorageConfig; + private HoodiePayloadConfig hoodiePayloadConfig; + private HoodieMetadataConfig metadataConfig; + + private EngineType engineType; + /** + * Use Spark engine by default. + */ protected HoodieWriteConfig(Properties props) { + this(EngineType.SPARK, props); + } + + protected HoodieWriteConfig(EngineType engineType, Properties props) { super(props); Properties newProps = new Properties(); newProps.putAll(props); + this.engineType = engineType; this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build(); this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build(); this.viewStorageConfig = clientSpecifiedViewStorageConfig; + this.hoodiePayloadConfig = HoodiePayloadConfig.newBuilder().fromProperties(newProps).build(); + this.metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(props).build(); } public static HoodieWriteConfig.Builder newBuilder() { @@ -177,12 +205,24 @@ public String getTableName() { return props.getProperty(TABLE_NAME); } + public String getPreCombineField() { + return props.getProperty(PRECOMBINE_FIELD_PROP); + } + + public String getWritePayloadClass() { + return props.getProperty(WRITE_PAYLOAD_CLASS); + } + + public String getKeyGeneratorClass() { + return props.getProperty(KEYGENERATOR_CLASS_PROP); + } + public Boolean shouldAutoCommit() { return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP)); } public Boolean shouldAssumeDatePartitioning() { - return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP)); + return metadataConfig.shouldAssumeDatePartitioning(); } public boolean shouldUseExternalSchemaTransformation() { @@ -217,6 +257,10 @@ public int getRollbackParallelism() { return Integer.parseInt(props.getProperty(ROLLBACK_PARALLELISM)); } + public int getFileListingParallelism() { + return metadataConfig.getFileListingParallelism(); + } + public boolean shouldRollbackUsingMarkers() { return Boolean.parseBoolean(props.getProperty(ROLLBACK_USING_MARKERS)); } @@ -282,6 +326,14 @@ public BulkInsertSortMode getBulkInsertSortMode() { return BulkInsertSortMode.valueOf(sortMode.toUpperCase()); } + public boolean isMergeDataValidationCheckEnabled() { + return Boolean.parseBoolean(props.getProperty(MERGE_DATA_VALIDATION_CHECK_ENABLED)); + } + + public EngineType getEngineType() { + return engineType; + } + /** * compaction properties. */ @@ -365,6 +417,23 @@ public Boolean getCompactionReverseLogReadEnabled() { return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); } + public boolean isInlineClustering() { + return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP)); + } + + public boolean isAsyncClusteringEnabled() { + return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY)); + } + + public boolean isClusteringEnabled() { + // TODO: future support async clustering + return isInlineClustering() || isAsyncClusteringEnabled(); + } + + public int getInlineClusterMaxCommits() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP)); + } + public String getPayloadClass() { return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP); } @@ -381,6 +450,45 @@ public Boolean shouldCleanBootstrapBaseFile() { return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED)); } + public String getClusteringUpdatesStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP); + } + + /** + * Clustering properties. + */ + public String getClusteringPlanStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS); + } + + public String getClusteringExecutionStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS); + } + + public long getClusteringMaxBytesInGroup() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP)); + } + + public long getClusteringSmallFileLimit() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT)); + } + + public int getClusteringMaxNumGroups() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS)); + } + + public long getClusteringTargetFileMaxBytes() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES)); + } + + public int getTargetPartitionsForClustering() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS)); + } + + public String getClusteringSortColumns() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY); + } + /** * index properties. */ @@ -587,6 +695,10 @@ public boolean isMetricsOn() { return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON)); } + public boolean isExecutorMetricsEnabled() { + return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.ENABLE_EXECUTOR_METRICS, "false")); + } + public MetricsReporterType getMetricsReporterType() { return MetricsReporterType.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE)); } @@ -721,6 +833,14 @@ public FileSystemViewStorageConfig getClientSpecifiedViewStorageConfig() { return clientSpecifiedViewStorageConfig; } + public HoodiePayloadConfig getPayloadConfig() { + return hoodiePayloadConfig; + } + + public HoodieMetadataConfig getMetadataConfig() { + return metadataConfig; + } + /** * Commit call back configs. */ @@ -768,18 +888,62 @@ public Long getMaxMemoryPerPartitionMerge() { return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP)); } + /** + * File listing metadata configs. + */ + public boolean useFileListingMetadata() { + return metadataConfig.useFileListingMetadata(); + } + + public boolean getFileListingMetadataVerify() { + return metadataConfig.validateFileListingMetadata(); + } + + public int getMetadataInsertParallelism() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP)); + } + + public int getMetadataCompactDeltaCommitMax() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_COMPACT_NUM_DELTA_COMMITS_PROP)); + } + + public boolean isMetadataAsyncClean() { + return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.METADATA_ASYNC_CLEAN_PROP)); + } + + public int getMetadataMaxCommitsToKeep() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP_PROP)); + } + + public int getMetadataMinCommitsToKeep() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP)); + } + + public int getMetadataCleanerCommitsRetained() { + return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP)); + } + public static class Builder { protected final Properties props = new Properties(); + protected EngineType engineType = EngineType.SPARK; private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; + private boolean isClusteringConfigSet = false; private boolean isMetricsConfigSet = false; private boolean isBootstrapConfigSet = false; private boolean isMemoryConfigSet = false; private boolean isViewConfigSet = false; private boolean isConsistencyGuardSet = false; private boolean isCallbackConfigSet = false; + private boolean isPayloadConfigSet = false; + private boolean isMetadataConfigSet = false; + + public Builder withEngineType(EngineType engineType) { + this.engineType = engineType; + return this; + } public Builder fromFile(File propertiesFile) throws IOException { try (FileReader reader = new FileReader(propertiesFile)) { @@ -822,6 +986,21 @@ public Builder forTable(String tableName) { return this; } + public Builder withPreCombineField(String preCombineField) { + props.setProperty(PRECOMBINE_FIELD_PROP, preCombineField); + return this; + } + + public Builder withWritePayLoad(String payload) { + props.setProperty(WRITE_PAYLOAD_CLASS, payload); + return this; + } + + public Builder withKeyGenerator(String keyGeneratorClass) { + props.setProperty(KEYGENERATOR_CLASS_PROP, keyGeneratorClass); + return this; + } + public Builder withTimelineLayoutVersion(int version) { props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version)); return this; @@ -897,6 +1076,12 @@ public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) { return this; } + public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) { + props.putAll(clusteringConfig.getProps()); + isClusteringConfigSet = true; + return this; + } + public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) { props.putAll(metricsConfig.getProps()); isMetricsConfigSet = true; @@ -915,13 +1100,20 @@ public Builder withBootstrapConfig(HoodieBootstrapConfig bootstrapConfig) { return this; } - public Builder withAutoCommit(boolean autoCommit) { - props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit)); + public Builder withPayloadConfig(HoodiePayloadConfig payloadConfig) { + props.putAll(payloadConfig.getProps()); + isPayloadConfigSet = true; + return this; + } + + public Builder withMetadataConfig(HoodieMetadataConfig metadataConfig) { + props.putAll(metadataConfig.getProps()); + isMetadataConfigSet = true; return this; } - public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) { - props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning)); + public Builder withAutoCommit(boolean autoCommit) { + props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit)); return this; } @@ -983,6 +1175,11 @@ public Builder withExternalSchemaTrasformation(boolean enabled) { return this; } + public Builder withMergeDataValidationCheckEnabled(boolean enabled) { + props.setProperty(MERGE_DATA_VALIDATION_CHECK_ENABLED, String.valueOf(enabled)); + return this; + } + public Builder withProperties(Properties properties) { this.props.putAll(properties); return this; @@ -995,8 +1192,13 @@ protected void setDefaults() { DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM, DEFAULT_ROLLBACK_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(KEYGENERATOR_CLASS_PROP), + KEYGENERATOR_CLASS_PROP, DEFAULT_KEYGENERATOR_CLASS); + setDefaultOnCondition(props, !props.containsKey(WRITE_PAYLOAD_CLASS), + WRITE_PAYLOAD_CLASS, DEFAULT_WRITE_PAYLOAD_CLASS); setDefaultOnCondition(props, !props.containsKey(ROLLBACK_USING_MARKERS), ROLLBACK_USING_MARKERS, DEFAULT_ROLLBACK_USING_MARKERS); setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP, @@ -1011,8 +1213,6 @@ protected void setDefaults() { DEFAULT_WRITE_STATUS_STORAGE_LEVEL); setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP), HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT); - setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP), - HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS); setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM, @@ -1032,12 +1232,16 @@ protected void setDefaults() { setDefaultOnCondition(props, !props.containsKey(AVRO_SCHEMA_VALIDATE), AVRO_SCHEMA_VALIDATE, DEFAULT_AVRO_SCHEMA_VALIDATE); setDefaultOnCondition(props, !props.containsKey(BULKINSERT_SORT_MODE), BULKINSERT_SORT_MODE, DEFAULT_BULKINSERT_SORT_MODE); + setDefaultOnCondition(props, !props.containsKey(MERGE_DATA_VALIDATION_CHECK_ENABLED), + MERGE_DATA_VALIDATION_CHECK_ENABLED, DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED); // Make sure the props is propagated - setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build()); setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isClusteringConfigSet, + HoodieClusteringConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isBootstrapConfigSet, HoodieBootstrapConfig.newBuilder().fromProperties(props).build()); @@ -1048,6 +1252,10 @@ protected void setDefaults() { ConsistencyGuardConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isCallbackConfigSet, HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isPayloadConfigSet, + HoodiePayloadConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isMetadataConfigSet, + HoodieMetadataConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION), EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION); @@ -1066,7 +1274,7 @@ public HoodieWriteConfig build() { setDefaults(); validate(); // Build WriteConfig at the end - HoodieWriteConfig config = new HoodieWriteConfig(props); + HoodieWriteConfig config = new HoodieWriteConfig(engineType, props); return config; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java new file mode 100644 index 0000000000000..bb6aaa24777cf --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java @@ -0,0 +1,31 @@ +/* + * 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.exception; + +public class HoodieClusteringException extends HoodieException { + + public HoodieClusteringException(String msg) { + super(msg); + } + + public HoodieClusteringException(String msg, Throwable e) { + super(msg, e); + } +} + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java new file mode 100644 index 0000000000000..68b62a5421706 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +public class HoodieClusteringUpdateException extends HoodieException { + public HoodieClusteringUpdateException(String msg) { + super(msg); + } + + public HoodieClusteringUpdateException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java index 8af72f351fff0..5e1f832b7f239 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java @@ -19,7 +19,7 @@ package org.apache.hudi.execution; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; @@ -71,7 +71,7 @@ public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime, public void consumeOneRecord(HoodieInsertValueGenResult payload) { final HoodieRecord insertPayload = payload.record; String partitionPath = insertPayload.getPartitionPath(); - HoodieWriteHandle handle = handles.get(partitionPath); + HoodieWriteHandle handle = handles.get(partitionPath); if (handle == null) { // If the records are sorted, this means that we encounter a new partition path // and the records for the previous partition path are all written, @@ -87,7 +87,7 @@ public void consumeOneRecord(HoodieInsertValueGenResult payload) { if (!handle.canWrite(payload.record)) { // Handle is full. Close the handle and add the WriteStatus - statuses.add(handle.close()); + statuses.addAll(handle.close()); // Open new handle handle = writeHandleFactory.create(config, instantTime, hoodieTable, insertPayload.getPartitionPath(), idPrefix, taskContextSupplier); @@ -108,8 +108,8 @@ public List getResult() { } private void closeOpenHandles() { - for (HoodieWriteHandle handle : handles.values()) { - statuses.add(handle.close()); + for (HoodieWriteHandle handle : handles.values()) { + statuses.addAll(handle.close()); } handles.clear(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java index b435c68de5e2b..cf544814cdba6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java @@ -20,7 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.LazyIterableIterator; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 6d04594cbab63..2e1915ff20431 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -21,7 +21,7 @@ import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIClass; import org.apache.hudi.PublicAPIMethod; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index ad7807b707dcb..55be8fc6598af 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -18,7 +18,7 @@ package org.apache.hudi.index; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java index 5c54dce31cd60..b4c83c141b2bc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java index 67ebadb2dd300..3a04c061a51a3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 0c590fe8818c4..c6ea7bab2ca7f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -20,8 +20,9 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieKey; @@ -30,9 +31,9 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.log.AppendResult; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieDataBlock; @@ -42,6 +43,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieAppendException; @@ -61,6 +63,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** * IO Operation to append data onto an existing file. @@ -69,48 +72,49 @@ public class HoodieAppendHandle extends private static final Logger LOG = LogManager.getLogger(HoodieAppendHandle.class); // This acts as the sequenceID for records written - private static AtomicLong recordIndex = new AtomicLong(1); + private static final AtomicLong RECORD_COUNTER = new AtomicLong(1); + private final String fileId; // Buffer for holding records in memory before they are flushed to disk - private List recordList = new ArrayList<>(); + private final List recordList = new ArrayList<>(); // Buffer for holding records (to be deleted) in memory before they are flushed to disk - private List keysToDelete = new ArrayList<>(); + private final List keysToDelete = new ArrayList<>(); + // Incoming records to be written to logs. + private final Iterator> recordItr; + // Writer to log into the file group's latest slice. + private Writer writer; - private Iterator> recordItr; + private final List statuses; // Total number of records written during an append private long recordsWritten = 0; // Total number of records deleted during an append private long recordsDeleted = 0; // Total number of records updated during an append private long updatedRecordsWritten = 0; + // Total number of new records inserted into the delta file + private long insertRecordsWritten = 0; + // Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk private long averageRecordSize = 0; - private HoodieLogFile currentLogFile; - private Writer writer; // Flag used to initialize some metadata private boolean doInit = true; // Total number of bytes written during this append phase (an estimation) private long estimatedNumberOfBytesWritten; - // Total number of bytes written to file - private long sizeInBytes = 0; // Number of records that must be written to meet the max block size for a log block private int numberOfRecords = 0; // Max block size to limit to for a log block - private int maxBlockSize = config.getLogFileDataBlockMaxSize(); + private final int maxBlockSize = config.getLogFileDataBlockMaxSize(); // Header metadata for a log block - private Map header = new HashMap<>(); - // Total number of new records inserted into the delta file - private long insertRecordsWritten = 0; - + private final Map header = new HashMap<>(); private SizeEstimator sizeEstimator; public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String partitionPath, String fileId, Iterator> recordItr, TaskContextSupplier taskContextSupplier) { super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); - writeStatus.setStat(new HoodieDeltaWriteStat()); this.fileId = fileId; this.recordItr = recordItr; sizeEstimator = new DefaultSizeEstimator(); + this.statuses = new ArrayList<>(); } public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, @@ -125,20 +129,32 @@ private void init(HoodieRecord record) { Option fileSlice = rtView.getLatestFileSlice(partitionPath, fileId); // Set the base commit time as the current instantTime for new inserts into log files String baseInstantTime; + String baseFile = ""; + List logFiles = new ArrayList<>(); if (fileSlice.isPresent()) { baseInstantTime = fileSlice.get().getBaseInstantTime(); + baseFile = fileSlice.get().getBaseFile().map(BaseFile::getFileName).orElse(""); + logFiles = fileSlice.get().getLogFiles().map(HoodieLogFile::getFileName).collect(Collectors.toList()); } else { baseInstantTime = instantTime; // This means there is no base data file, start appending to a new log file fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId)); - LOG.info("New InsertHandle for partition :" + partitionPath); + LOG.info("New AppendHandle for partition :" + partitionPath); } - writeStatus.getStat().setPrevCommit(baseInstantTime); + + // Prepare the first write status + writeStatus.setStat(new HoodieDeltaWriteStat()); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); - writeStatus.getStat().setPartitionPath(partitionPath); - writeStatus.getStat().setFileId(fileId); averageRecordSize = sizeEstimator.sizeEstimate(record); + + HoodieDeltaWriteStat deltaWriteStat = (HoodieDeltaWriteStat) writeStatus.getStat(); + deltaWriteStat.setPrevCommit(baseInstantTime); + deltaWriteStat.setPartitionPath(partitionPath); + deltaWriteStat.setFileId(fileId); + deltaWriteStat.setBaseFile(baseFile); + deltaWriteStat.setLogFiles(logFiles); + try { //save hoodie partition meta in the partition path HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime, @@ -148,34 +164,29 @@ private void init(HoodieRecord record) { // Since the actual log file written to can be different based on when rollover happens, we use the // base file to denote some log appends happened on a slice. writeToken will still fence concurrent // writers. + // https://issues.apache.org/jira/browse/HUDI-1517 createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension())); this.writer = createLogWriter(fileSlice, baseInstantTime); - this.currentLogFile = writer.getLogFile(); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion()); - ((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize()); } catch (Exception e) { LOG.error("Error in update task at commit " + instantTime, e); writeStatus.setGlobalError(e); throw new HoodieUpsertException("Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit " + instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + partitionPath, e); } - Path path = partitionPath.length() == 0 ? new Path(writer.getLogFile().getFileName()) - : new Path(partitionPath, writer.getLogFile().getFileName()); - writeStatus.getStat().setPath(path.toString()); doInit = false; } } private Option getIndexedRecord(HoodieRecord hoodieRecord) { - Option recordMetadata = hoodieRecord.getData().getMetadata(); + Option> recordMetadata = hoodieRecord.getData().getMetadata(); try { Option avroRecord = hoodieRecord.getData().getInsertValue(writerSchema); if (avroRecord.isPresent()) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get())); String seqId = - HoodieRecord.generateSequenceId(instantTime, getPartitionId(), recordIndex.getAndIncrement()); + HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(), hoodieRecord.getPartitionPath(), fileId); HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId); @@ -203,6 +214,105 @@ private Option getIndexedRecord(HoodieRecord hoodieRecord) { return Option.empty(); } + private void initNewStatus() { + HoodieDeltaWriteStat prevStat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); + // Make a new write status and copy basic fields over. + HoodieDeltaWriteStat stat = new HoodieDeltaWriteStat(); + stat.setFileId(fileId); + stat.setPartitionPath(partitionPath); + stat.setPrevCommit(prevStat.getPrevCommit()); + stat.setBaseFile(prevStat.getBaseFile()); + stat.setLogFiles(new ArrayList<>(prevStat.getLogFiles())); + + this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), + !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); + this.writeStatus.setFileId(fileId); + this.writeStatus.setPartitionPath(partitionPath); + this.writeStatus.setStat(stat); + } + + private String makeFilePath(HoodieLogFile logFile) { + return partitionPath.length() == 0 + ? new Path(logFile.getFileName()).toString() + : new Path(partitionPath, logFile.getFileName()).toString(); + } + + private void resetWriteCounts() { + recordsWritten = 0; + updatedRecordsWritten = 0; + insertRecordsWritten = 0; + recordsDeleted = 0; + } + + private void updateWriteCounts(HoodieDeltaWriteStat stat, AppendResult result) { + stat.setNumWrites(recordsWritten); + stat.setNumUpdateWrites(updatedRecordsWritten); + stat.setNumInserts(insertRecordsWritten); + stat.setNumDeletes(recordsDeleted); + stat.setTotalWriteBytes(result.size()); + } + + private void accumulateWriteCounts(HoodieDeltaWriteStat stat, AppendResult result) { + stat.setNumWrites(stat.getNumWrites() + recordsWritten); + stat.setNumUpdateWrites(stat.getNumUpdateWrites() + updatedRecordsWritten); + stat.setNumInserts(stat.getNumInserts() + insertRecordsWritten); + stat.setNumDeletes(stat.getNumDeletes() + recordsDeleted); + stat.setTotalWriteBytes(stat.getTotalWriteBytes() + result.size()); + } + + private void updateWriteStat(HoodieDeltaWriteStat stat, AppendResult result) { + stat.setPath(makeFilePath(result.logFile())); + stat.setLogOffset(result.offset()); + stat.setLogVersion(result.logFile().getLogVersion()); + if (!stat.getLogFiles().contains(result.logFile().getFileName())) { + stat.addLogFiles(result.logFile().getFileName()); + } + } + + private void updateRuntimeStats(HoodieDeltaWriteStat stat) { + RuntimeStats runtimeStats = new RuntimeStats(); + runtimeStats.setTotalUpsertTime(timer.endTimer()); + stat.setRuntimeStats(runtimeStats); + } + + private void accumulateRuntimeStats(HoodieDeltaWriteStat stat) { + RuntimeStats runtimeStats = stat.getRuntimeStats(); + assert runtimeStats != null; + runtimeStats.setTotalUpsertTime(runtimeStats.getTotalUpsertTime() + timer.endTimer()); + } + + private void updateWriteStatus(HoodieDeltaWriteStat stat, AppendResult result) { + updateWriteStat(stat, result); + updateWriteCounts(stat, result); + updateRuntimeStats(stat); + statuses.add(this.writeStatus); + } + + private void processAppendResult(AppendResult result) { + HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); + + if (stat.getPath() == null) { + // first time writing to this log block. + updateWriteStatus(stat, result); + } else if (stat.getPath().endsWith(result.logFile().getFileName())) { + // append/continued writing to the same log file + stat.setLogOffset(Math.min(stat.getLogOffset(), result.offset())); + accumulateWriteCounts(stat, result); + accumulateRuntimeStats(stat); + } else { + // written to a newer log file, due to rollover/otherwise. + initNewStatus(); + stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); + updateWriteStatus(stat, result); + } + + resetWriteCounts(); + assert stat.getRuntimeStats() != null; + LOG.info(String.format("AppendHandle for partitionPath %s filePath %s, took %d ms.", partitionPath, + stat.getPath(), stat.getRuntimeStats().getTotalUpsertTime())); + timer.startTimer(); + } + public void doAppend() { while (recordItr.hasNext()) { HoodieRecord record = recordItr.next(); @@ -210,24 +320,30 @@ public void doAppend() { flushToDiskIfRequired(record); writeToBuffer(record); } - doAppend(header); + appendDataAndDeleteBlocks(header); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; } - private void doAppend(Map header) { + private void appendDataAndDeleteBlocks(Map header) { try { header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchemaWithMetafields.toString()); + List blocks = new ArrayList<>(2); if (recordList.size() > 0) { - writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); - recordList.clear(); + blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header)); } if (keysToDelete.size() > 0) { - writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header)); + blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header)); + } + + if (blocks.size() > 0) { + AppendResult appendResult = writer.appendBlocks(blocks); + processAppendResult(appendResult); + recordList.clear(); keysToDelete.clear(); } } catch (Exception e) { - throw new HoodieAppendException("Failed while appending records to " + currentLogFile.getPath(), e); + throw new HoodieAppendException("Failed while appending records to " + writer.getLogFile().getPath(), e); } } @@ -239,7 +355,7 @@ public boolean canWrite(HoodieRecord record) { @Override public void write(HoodieRecord record, Option insertValue) { - Option recordMetadata = record.getData().getMetadata(); + Option> recordMetadata = record.getData().getMetadata(); try { init(record); flushToDiskIfRequired(record); @@ -253,48 +369,34 @@ public void write(HoodieRecord record, Option insertValue) { } @Override - public WriteStatus close() { + public List close() { try { // flush any remaining records to disk - doAppend(header); - + appendDataAndDeleteBlocks(header); if (writer != null) { - sizeInBytes = writer.getCurrentSize(); writer.close(); - } - HoodieWriteStat stat = writeStatus.getStat(); - stat.setFileId(this.fileId); - stat.setNumWrites(recordsWritten); - stat.setNumUpdateWrites(updatedRecordsWritten); - stat.setNumInserts(insertRecordsWritten); - stat.setNumDeletes(recordsDeleted); - stat.setTotalWriteBytes(estimatedNumberOfBytesWritten); - stat.setFileSizeInBytes(sizeInBytes); - stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords()); - RuntimeStats runtimeStats = new RuntimeStats(); - runtimeStats.setTotalUpsertTime(timer.endTimer()); - stat.setRuntimeStats(runtimeStats); - - LOG.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(), - stat.getFileId(), runtimeStats.getTotalUpsertTime())); - - return writeStatus; + // update final size, once for all log files + for (WriteStatus status: statuses) { + long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath())); + status.getStat().setFileSizeInBytes(logFileSize); + } + } + return statuses; } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } } - @Override - public WriteStatus getWriteStatus() { - return writeStatus; - } - @Override public IOType getIOType() { return IOType.APPEND; } + public List writeStatuses() { + return statuses; + } + private Writer createLogWriter(Option fileSlice, String baseCommitTime) throws IOException, InterruptedException { Option latestLogFile = fileSlice.get().getLatestLogFile(); @@ -303,9 +405,10 @@ private Writer createLogWriter(Option fileSlice, String baseCommitTim .onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath)) .withFileId(fileId).overBaseCommit(baseCommitTime) .withLogVersion(latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION)) + .withFileSize(latestLogFile.map(HoodieLogFile::getFileSize).orElse(0L)) .withSizeThreshold(config.getLogFileMaxSize()).withFs(fs) - .withLogWriteToken(latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken)) .withRolloverLogWriteToken(writeToken) + .withLogWriteToken(latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken)) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); } @@ -340,7 +443,7 @@ private void flushToDiskIfRequired(HoodieRecord record) { // avg of new and old LOG.info("AvgRecordSize => " + averageRecordSize); averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2; - doAppend(header); + appendDataAndDeleteBlocks(header); estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords; numberOfRecords = 0; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java index b2eaedd53ce7d..d91f5c805203e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 6a8e7735093d5..8fbd1be247217 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -20,7 +20,7 @@ import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; @@ -44,7 +44,9 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Map; public class HoodieCreateHandle extends HoodieWriteHandle { @@ -162,11 +164,6 @@ public void write() { } } - @Override - public WriteStatus getWriteStatus() { - return writeStatus; - } - @Override public IOType getIOType() { return IOType.CREATE; @@ -176,9 +173,8 @@ public IOType getIOType() { * Performs actions to durably, persist the current changes and returns a WriteStatus object. */ @Override - public WriteStatus close() { - LOG - .info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten); + public List close() { + LOG.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten); try { fileWriter.close(); @@ -203,7 +199,7 @@ public WriteStatus close() { LOG.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalCreateTime())); - return writeStatus; + return Collections.singletonList(writeStatus); } catch (IOException e) { throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java index 9194fc042f988..a5bc6b2fe55d2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java @@ -29,9 +29,7 @@ import org.apache.hadoop.fs.Path; -import java.util.Iterator; - -import scala.Tuple2; +import java.util.stream.Stream; /** * {@link HoodieRecordLocation} fetch handle for all records from {@link HoodieBaseFile} of interest. @@ -48,10 +46,10 @@ public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable> locations() { + public Stream> locations() { HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight(); return ParquetUtils.fetchRecordKeyPartitionPathFromParquet(hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream() - .map(entry -> new Tuple2<>(entry, - new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))).iterator(); + .map(entry -> Pair.of(entry, + new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index be324a3c5081a..4f5b82a67d46d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -34,8 +34,11 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.table.HoodieTable; @@ -47,8 +50,10 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -222,7 +227,8 @@ public void write(GenericRecord oldRecord) { HoodieRecord hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key)); try { Option combinedAvroRecord = - hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchemaWithMetafields : writerSchema); + hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchemaWithMetafields : writerSchema, + config.getPayloadConfig().getProps()); if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) { /* * ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully write the the combined new @@ -241,14 +247,11 @@ public void write(GenericRecord oldRecord) { if (copyOldRecord) { // this should work as it is, since this is an existing record - String errMsg = "Failed to merge old record into new file for key " + key + " from old file " + getOldFilePath() - + " to new file " + newFilePath + " with writerSchema " + writerSchemaWithMetafields.toString(true); try { fileWriter.writeAvro(key, oldRecord); - } catch (ClassCastException e) { - LOG.debug("Old record is " + oldRecord); - throw new HoodieUpsertException(errMsg, e); } catch (IOException | RuntimeException e) { + String errMsg = String.format("Failed to merge old record into new file for key %s from old file %s to new file %s with writerSchema %s", + key, getOldFilePath(), newFilePath, writerSchemaWithMetafields.toString(true)); LOG.debug("Old record is " + oldRecord); throw new HoodieUpsertException(errMsg, e); } @@ -257,7 +260,7 @@ public void write(GenericRecord oldRecord) { } @Override - public WriteStatus close() { + public List close() { try { // write out any pending records (this can happen when inserts are turned into updates) Iterator> newRecordsItr = (keyToNewRecords instanceof ExternalSpillableMap) @@ -295,22 +298,41 @@ public WriteStatus close() { runtimeStats.setTotalUpsertTime(timer.endTimer()); stat.setRuntimeStats(runtimeStats); + performMergeDataValidationCheck(writeStatus); + LOG.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime())); - return writeStatus; + return Collections.singletonList(writeStatus); } catch (IOException e) { throw new HoodieUpsertException("Failed to close UpdateHandle", e); } } - public Path getOldFilePath() { - return oldFilePath; + public void performMergeDataValidationCheck(WriteStatus writeStatus) { + if (!config.isMergeDataValidationCheckEnabled()) { + return; + } + + long oldNumWrites = 0; + try { + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), oldFilePath); + oldNumWrites = reader.getTotalRecords(); + } catch (IOException e) { + throw new HoodieUpsertException("Failed to check for merge data validation", e); + } + + if ((writeStatus.getStat().getNumWrites() + writeStatus.getStat().getNumDeletes()) < oldNumWrites) { + throw new HoodieCorruptedDataException( + String.format("Record write count decreased for file: %s, Partition Path: %s (%s:%d + %d < %s:%d)", + writeStatus.getFileId(), writeStatus.getPartitionPath(), + instantTime, writeStatus.getStat().getNumWrites(), writeStatus.getStat().getNumDeletes(), + FSUtils.getCommitTime(oldFilePath.toString()), oldNumWrites)); + } } - @Override - public WriteStatus getWriteStatus() { - return writeStatus; + public Path getOldFilePath() { + return oldFilePath; } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java index 71610b1aa9c2f..1eee1046337a6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java @@ -19,7 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.PriorityQueue; import java.util.Queue; @@ -101,7 +102,7 @@ public void write(GenericRecord oldRecord) { } @Override - public WriteStatus close() { + public List close() { // write out any pending records (this can happen when inserts are turned into updates) newRecordKeysSorted.stream().forEach(key -> { try { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 85898bccc7052..d2e538e2b661c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -45,6 +45,8 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collections; +import java.util.List; /** * Base class for all write operations logically performed at the file group level. @@ -56,7 +58,7 @@ public abstract class HoodieWriteHandle protected final Schema writerSchema; protected final Schema writerSchemaWithMetafields; protected HoodieTimer timer; - protected final WriteStatus writeStatus; + protected WriteStatus writeStatus; protected final String partitionPath; protected final String fileId; protected final String writeToken; @@ -167,9 +169,15 @@ protected GenericRecord rewriteRecord(GenericRecord record) { return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields); } - public abstract WriteStatus close(); + public abstract List close(); - public abstract WriteStatus getWriteStatus(); + public List writeStatuses() { + return Collections.singletonList(writeStatus); + } + + public String getPartitionPath() { + return partitionPath; + } public abstract IOType getIOType(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java index 03719157fd998..6b5891c86b2f4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java @@ -18,8 +18,8 @@ package org.apache.hudi.io; -import org.apache.hudi.client.common.EngineProperty; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.util.Option; import java.util.Properties; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java index c66442a48d5ad..d2923d4478cd9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 5f4eec0887003..e1e8c5e40273d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -19,9 +19,9 @@ package org.apache.hudi.io.storage; import org.apache.hudi.avro.HoodieAvroWriteSupport; -import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 3684f9d4cd931..1358c35be2e59 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -19,8 +19,8 @@ package org.apache.hudi.io.storage; import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java index 166e2bc42ab98..d1c695a45b012 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroWriteSupport; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java index 6266fd15c6b84..724cabdc73964 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java @@ -88,7 +88,7 @@ public String getPartitionPath(GenericRecord record) { try { partitionPath.append(new TimestampBasedAvroKeyGenerator(config, partitionPathField).getPartitionPath(record)); } catch (IOException e) { - throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class"); + throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class", e); } break; default: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java new file mode 100644 index 0000000000000..1f76a5e30b803 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -0,0 +1,467 @@ +/* + * 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.metadata; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; +import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; + +/** + * Writer implementation backed by an internal hudi table. Partition and file listing are saved within an internal MOR table + * called Metadata Table. This table is created by listing files and partitions (first time) + * and kept in sync using the instants on the main dataset. + */ +public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMetadataWriter { + + private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); + + protected HoodieWriteConfig metadataWriteConfig; + protected HoodieWriteConfig datasetWriteConfig; + protected String tableName; + + protected HoodieBackedTableMetadata metadata; + protected HoodieTableMetaClient metaClient; + protected Option metrics; + protected boolean enabled; + protected SerializableConfiguration hadoopConf; + protected final transient HoodieEngineContext engineContext; + + protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { + this.datasetWriteConfig = writeConfig; + this.engineContext = engineContext; + this.hadoopConf = new SerializableConfiguration(hadoopConf); + + if (writeConfig.useFileListingMetadata()) { + this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX; + this.metadataWriteConfig = createMetadataWriteConfig(writeConfig); + enabled = true; + + // Inline compaction and auto clean is required as we dont expose this table outside + ValidationUtils.checkArgument(!this.metadataWriteConfig.isAutoClean(), "Cleaning is controlled internally for Metadata table."); + ValidationUtils.checkArgument(!this.metadataWriteConfig.isInlineCompaction(), "Compaction is controlled internally for metadata table."); + // Metadata Table cannot have metadata listing turned on. (infinite loop, much?) + ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table"); + ValidationUtils.checkArgument(!this.metadataWriteConfig.useFileListingMetadata(), "File listing cannot be used for Metadata Table"); + + initRegistry(); + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf, datasetWriteConfig.getBasePath()); + initialize(engineContext, datasetMetaClient); + if (enabled) { + // This is always called even in case the table was created for the first time. This is because + // initFromFilesystem() does file listing and hence may take a long time during which some new updates + // may have occurred on the table. Hence, calling this always ensures that the metadata is brought in sync + // with the active timeline. + HoodieTimer timer = new HoodieTimer().startTimer(); + syncFromInstants(datasetMetaClient); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.SYNC_STR, timer.endTimer())); + } + } else { + enabled = false; + this.metrics = Option.empty(); + } + } + + protected abstract void initRegistry(); + + /** + * Create a {@code HoodieWriteConfig} to use for the Metadata Table. + * + * @param writeConfig {@code HoodieWriteConfig} of the main dataset writer + */ + private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfig) { + int parallelism = writeConfig.getMetadataInsertParallelism(); + + // Create the write config for the metadata table by borrowing options from the main write config. + HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() + .withConsistencyCheckEnabled(writeConfig.getConsistencyGuardConfig().isConsistencyCheckEnabled()) + .withInitialConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getInitialConsistencyCheckIntervalMs()) + .withMaxConsistencyCheckIntervalMs(writeConfig.getConsistencyGuardConfig().getMaxConsistencyCheckIntervalMs()) + .withMaxConsistencyChecks(writeConfig.getConsistencyGuardConfig().getMaxConsistencyChecks()) + .build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .withAutoCommit(true) + .withAvroSchemaValidate(true) + .withEmbeddedTimelineServerEnabled(false) + .withPath(HoodieTableMetadata.getMetadataTableBasePath(writeConfig.getBasePath())) + .withSchema(HoodieMetadataRecord.getClassSchema().toString()) + .forTable(tableName) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withAsyncClean(writeConfig.isMetadataAsyncClean()) + // we will trigger cleaning manually, to control the instant times + .withAutoClean(false) + .withCleanerParallelism(parallelism) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) + .archiveCommitsWith(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMetadataMaxCommitsToKeep()) + // we will trigger compaction manually, to control the instant times + .withInlineCompaction(false) + .withMaxNumDeltaCommitsBeforeCompaction(writeConfig.getMetadataCompactDeltaCommitMax()).build()) + .withParallelism(parallelism, parallelism) + .withDeleteParallelism(parallelism) + .withRollbackParallelism(parallelism) + .withFinalizeWriteParallelism(parallelism); + + if (writeConfig.isMetricsOn()) { + HoodieMetricsConfig.Builder metricsConfig = HoodieMetricsConfig.newBuilder() + .withReporterType(writeConfig.getMetricsReporterType().toString()) + .withExecutorMetrics(writeConfig.isExecutorMetricsEnabled()) + .on(true); + switch (writeConfig.getMetricsReporterType()) { + case GRAPHITE: + metricsConfig.onGraphitePort(writeConfig.getGraphiteServerPort()) + .toGraphiteHost(writeConfig.getGraphiteServerHost()) + .usePrefix(writeConfig.getGraphiteMetricPrefix()); + break; + case JMX: + metricsConfig.onJmxPort(writeConfig.getJmxPort()) + .toJmxHost(writeConfig.getJmxHost()); + break; + case DATADOG: + // TODO: + break; + case CONSOLE: + case INMEMORY: + break; + default: + throw new HoodieMetadataException("Unsupported Metrics Reporter type " + writeConfig.getMetricsReporterType()); + } + + builder.withMetricsConfig(metricsConfig.build()); + } + + return builder.build(); + } + + public HoodieWriteConfig getWriteConfig() { + return metadataWriteConfig; + } + + public HoodieBackedTableMetadata metadata() { + return metadata; + } + + /** + * Initialize the metadata table if it does not exist. Update the metadata to bring it in sync with the file system. + * + * This can happen in two ways: + * 1. If the metadata table did not exist, then file and partition listing is used + * 2. If the metadata table exists, the instants from active timeline are read in order and changes applied + * + * The above logic has been chosen because it is faster to perform #1 at scale rather than read all the Instants + * which are large in size (AVRO or JSON encoded and not compressed) and incur considerable IO for de-serialization + * and decoding. + */ + protected abstract void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient); + + protected void initTableMetadata() { + try { + if (this.metadata != null) { + this.metadata.close(); + } + this.metadata = new HoodieBackedTableMetadata(engineContext, datasetWriteConfig.getMetadataConfig(), + datasetWriteConfig.getBasePath(), datasetWriteConfig.getSpillableMapBasePath()); + this.metaClient = metadata.getMetaClient(); + } catch (Exception e) { + throw new HoodieException("Error initializing metadata table for reads", e); + } + } + + protected void bootstrapIfNeeded(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + boolean exists = datasetMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); + if (!exists) { + // Initialize for the first time by listing partitions and files directly from the file system + bootstrapFromFilesystem(engineContext, datasetMetaClient); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); + } + } + + /** + * Initialize the Metadata Table by listing files and partitions from the file system. + * + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + */ + private void bootstrapFromFilesystem(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) throws IOException { + ValidationUtils.checkState(enabled, "Metadata table cannot be initialized as it is not enabled"); + + // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit + // Otherwise, we use the timestamp of the instant which does not have any non-completed instants before it. + Option latestInstant = Option.empty(); + boolean foundNonComplete = false; + for (HoodieInstant instant : datasetMetaClient.getActiveTimeline().getInstants().collect(Collectors.toList())) { + if (!instant.isCompleted()) { + foundNonComplete = true; + } else if (!foundNonComplete) { + latestInstant = Option.of(instant); + } + } + + String createInstantTime = latestInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); + + HoodieTableMetaClient.initTableType(hadoopConf.get(), metadataWriteConfig.getBasePath(), + HoodieTableType.MERGE_ON_READ, tableName, "archived", HoodieMetadataPayload.class.getName(), + HoodieFileFormat.HFILE.toString()); + initTableMetadata(); + + // List all partitions in the basePath of the containing dataset + LOG.info("Initializing metadata table by using file listings in " + datasetWriteConfig.getBasePath()); + Map> partitionToFileStatus = getPartitionsToFilesMapping(datasetMetaClient); + + // Create a HoodieCommitMetadata with writeStats for all discovered files + int[] stats = {0}; + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + + partitionToFileStatus.forEach((partition, statuses) -> { + // Filter the statuses to only include files which were created before or on createInstantTime + statuses.stream().filter(status -> { + String filename = status.getPath().getName(); + return !HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(filename), HoodieTimeline.GREATER_THAN, + createInstantTime); + }).forEach(status -> { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPath(partition + Path.SEPARATOR + status.getPath().getName()); + writeStat.setPartitionPath(partition); + writeStat.setTotalWriteBytes(status.getLen()); + commitMetadata.addWriteStat(partition, writeStat); + stats[0] += 1; + }); + + // If the partition has no files then create a writeStat with no file path + if (commitMetadata.getWriteStats(partition) == null) { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPartitionPath(partition); + commitMetadata.addWriteStat(partition, writeStat); + } + }); + + LOG.info("Committing " + partitionToFileStatus.size() + " partitions and " + stats[0] + " files to metadata"); + update(commitMetadata, createInstantTime); + } + + /** + * Function to find hoodie partitions and list files in them in parallel. + * + * @param datasetMetaClient + * @return Map of partition names to a list of FileStatus for all the files in the partition + */ + private Map> getPartitionsToFilesMapping(HoodieTableMetaClient datasetMetaClient) { + List pathsToList = new LinkedList<>(); + pathsToList.add(new Path(datasetWriteConfig.getBasePath())); + + Map> partitionToFileStatus = new HashMap<>(); + final int fileListingParallelism = metadataWriteConfig.getFileListingParallelism(); + SerializableConfiguration conf = new SerializableConfiguration(datasetMetaClient.getHadoopConf()); + + while (!pathsToList.isEmpty()) { + int listingParallelism = Math.min(fileListingParallelism, pathsToList.size()); + // List all directories in parallel + List> dirToFileListing = engineContext.map(pathsToList, path -> { + FileSystem fs = path.getFileSystem(conf.get()); + return Pair.of(path, fs.listStatus(path)); + }, listingParallelism); + pathsToList.clear(); + + // If the listing reveals a directory, add it to queue. If the listing reveals a hoodie partition, add it to + // the results. + dirToFileListing.forEach(p -> { + List filesInDir = Arrays.stream(p.getRight()).parallel() + .filter(fs -> !fs.getPath().getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) + .collect(Collectors.toList()); + + if (p.getRight().length > filesInDir.size()) { + // Is a partition. Add all data files to result. + String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), p.getLeft()); + partitionToFileStatus.put(partitionName, filesInDir); + } else { + // Add sub-dirs to the queue + pathsToList.addAll(Arrays.stream(p.getRight()) + .filter(fs -> fs.isDirectory() && !fs.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) + .map(fs -> fs.getPath()) + .collect(Collectors.toList())); + } + }); + } + + return partitionToFileStatus; + } + + /** + * Sync the Metadata Table from the instants created on the dataset. + * + * @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset + */ + private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { + ValidationUtils.checkState(enabled, "Metadata table cannot be synced as it is not enabled"); + // (re) init the metadata for reading. + initTableMetadata(); + try { + List instantsToSync = metadata.findInstantsToSync(); + if (instantsToSync.isEmpty()) { + return; + } + + LOG.info("Syncing " + instantsToSync.size() + " instants to metadata table: " + instantsToSync); + + // Read each instant in order and sync it to metadata table + for (HoodieInstant instant : instantsToSync) { + LOG.info("Syncing instant " + instant + " to metadata table"); + + Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, metadata.getSyncedInstantTime()); + if (records.isPresent()) { + commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp()); + } + } + initTableMetadata(); + } catch (IOException ioe) { + throw new HoodieIOException("Unable to sync instants from data to metadata table.", ioe); + } + } + + /** + * Update from {@code HoodieCommitMetadata}. + * + * @param commitMetadata {@code HoodieCommitMetadata} + * @param instantTime Timestamp at which the commit was performed + */ + @Override + public void update(HoodieCommitMetadata commitMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieCleanerPlan}. + * + * @param cleanerPlan {@code HoodieCleanerPlan} + * @param instantTime Timestamp at which the clean plan was generated + */ + @Override + public void update(HoodieCleanerPlan cleanerPlan, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanerPlan, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieCleanMetadata}. + * + * @param cleanMetadata {@code HoodieCleanMetadata} + * @param instantTime Timestamp at which the clean was completed + */ + @Override + public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieRestoreMetadata}. + * + * @param restoreMetadata {@code HoodieRestoreMetadata} + * @param instantTime Timestamp at which the restore was performed + */ + @Override + public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(restoreMetadata, instantTime, metadata.getSyncedInstantTime()); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + /** + * Update from {@code HoodieRollbackMetadata}. + * + * @param rollbackMetadata {@code HoodieRollbackMetadata} + * @param instantTime Timestamp at which the rollback was performed + */ + @Override + public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) { + if (enabled) { + List records = HoodieTableMetadataUtil.convertMetadataToRecords(rollbackMetadata, instantTime, metadata.getSyncedInstantTime()); + commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime); + } + } + + @Override + public void close() throws Exception { + if (metadata != null) { + metadata.close(); + } + } + + /** + * Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit. + * + */ + protected abstract void commit(List records, String partitionName, String instantTime); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java new file mode 100644 index 0000000000000..02c5b9e646ad0 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; + +import java.io.Serializable; + +/** + * Interface that supports updating metadata for a given table, as actions complete. + */ +public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { + + void update(HoodieCommitMetadata commitMetadata, String instantTime); + + void update(HoodieCleanerPlan cleanerPlan, String instantTime); + + void update(HoodieCleanMetadata cleanMetadata, String instantTime); + + void update(HoodieRestoreMetadata restoreMetadata, String instantTime); + + void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 292039b83ad41..c8c112fc203ff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -37,6 +37,7 @@ public class HoodieMetrics { public String cleanTimerName = null; public String commitTimerName = null; public String deltaCommitTimerName = null; + public String replaceCommitTimerName = null; public String finalizeTimerName = null; public String compactionTimerName = null; public String indexTimerName = null; @@ -48,6 +49,7 @@ public class HoodieMetrics { private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; private Timer compactionTimer = null; + private Timer clusteringTimer = null; private Timer indexTimer = null; public HoodieMetrics(HoodieWriteConfig config, String tableName) { @@ -59,6 +61,7 @@ public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION); this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION); this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION); + this.replaceCommitTimerName = getMetricsName("timer", HoodieTimeline.REPLACE_COMMIT_ACTION); this.finalizeTimerName = getMetricsName("timer", "finalize"); this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); this.indexTimerName = getMetricsName("timer", "index"); @@ -83,6 +86,13 @@ public Timer.Context getCompactionCtx() { return compactionTimer == null ? null : compactionTimer.time(); } + public Timer.Context getClusteringCtx() { + if (config.isMetricsOn() && clusteringTimer == null) { + clusteringTimer = createTimer(replaceCommitTimerName); + } + return clusteringTimer == null ? null : clusteringTimer.time(); + } + public Timer.Context getCleanCtx() { if (config.isMetricsOn() && cleanTimer == null) { cleanTimer = createTimer(cleanTimerName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 0413f75ce066d..74ffdfc653173 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -24,13 +24,16 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.client.common.TaskContextSupplier; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.ConsistencyGuard; import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility; import org.apache.hudi.common.fs.ConsistencyGuardConfig; @@ -47,6 +50,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; @@ -60,6 +64,7 @@ import org.apache.hudi.exception.HoodieInsertException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.log4j.LogManager; @@ -90,17 +95,25 @@ public abstract class HoodieTable implem protected final HoodieWriteConfig config; protected final HoodieTableMetaClient metaClient; protected final HoodieIndex index; - private SerializableConfiguration hadoopConfiguration; - private transient FileSystemViewManager viewManager; - protected final TaskContextSupplier taskContextSupplier; + private final HoodieTableMetadata metadata; + + private transient FileSystemViewManager viewManager; + protected final transient HoodieEngineContext context; protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { this.config = config; this.hadoopConfiguration = context.getHadoopConf(); - this.viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, - config.getViewStorageConfig()); + this.context = context; + + // disable reuse of resources, given there is no close() called on the executors ultimately + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps()) + .enableReuse(false).build(); + this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(), + FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR); + + this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), () -> metadata); this.metaClient = metaClient; this.index = getIndex(config, context); this.taskContextSupplier = context.getTaskContextSupplier(); @@ -110,7 +123,7 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo private synchronized FileSystemViewManager getViewManager() { if (null == viewManager) { - viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig()); + viewManager = FileSystemViewManager.createViewManager(getContext(), config.getMetadataConfig(), config.getViewStorageConfig(), () -> metadata); } return viewManager; } @@ -157,6 +170,15 @@ public abstract HoodieWriteMetadata bulkInsert(HoodieEngineContext context, S */ public abstract HoodieWriteMetadata delete(HoodieEngineContext context, String instantTime, K keys); + /** + * Deletes all data of partitions. + * @param context HoodieEngineContext + * @param instantTime Instant Time for the action + * @param partitions {@link List} of partition to be deleted + * @return HoodieWriteMetadata + */ + public abstract HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions); + /** * Upserts the given prepared records into the Hoodie table, at the supplied instantTime. *

@@ -205,6 +227,17 @@ public abstract HoodieWriteMetadata bulkInsertPrepped(HoodieEngineContext con */ public abstract HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, I records); + /** + * Delete all the existing records of the Hoodie table and inserts the specified new records into Hoodie table at the supplied instantTime, + * for the partition paths contained in input records. + * + * @param context HoodieEngineContext + * @param instantTime Instant time for the replace action + * @param records input records + * @return HoodieWriteMetadata + */ + public abstract HoodieWriteMetadata insertOverwriteTable(HoodieEngineContext context, String instantTime, I records); + public HoodieWriteConfig getConfig() { return config; } @@ -326,6 +359,27 @@ public abstract Option scheduleCompaction(HoodieEngineCont public abstract HoodieWriteMetadata compact(HoodieEngineContext context, String compactionInstantTime); + + /** + * Schedule clustering for the instant time. + * + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling clustering + * @param extraMetadata additional metadata to write into plan + * @return HoodieClusteringPlan, if there is enough data for clustering. + */ + public abstract Option scheduleClustering(HoodieEngineContext context, + String instantTime, + Option> extraMetadata); + + /** + * Execute Clustering on the table. Clustering re-arranges the data so that it is optimized for data access. + * + * @param context HoodieEngineContext + * @param clusteringInstantTime Instant Time + */ + public abstract HoodieWriteMetadata cluster(HoodieEngineContext context, String clusteringInstantTime); + /** * Perform metadata/full bootstrap of a Hudi table. * @param context HoodieEngineContext @@ -392,6 +446,7 @@ public void finalizeWrite(HoodieEngineContext context, String instantTs, List>> invalidFilesByPartition) { // Now delete partially written files + context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation"); context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> { final FileSystem fileSystem = metaClient.getFs(); LOG.info("Deleting invalid data files=" + partitionWithFileList); @@ -449,7 +504,7 @@ protected void reconcileAgainstMarkers(HoodieEngineContext context, if (!invalidDataPaths.isEmpty()) { LOG.info("Removing duplicate data files created due to spark retries before committing. Paths=" + invalidDataPaths); Map>> invalidPathsByPartition = invalidDataPaths.stream() - .map(dp -> Pair.of(new Path(dp).getParent().toString(), new Path(basePath, dp).toString())) + .map(dp -> Pair.of(new Path(basePath, dp).getParent().toString(), new Path(basePath, dp).toString())) .collect(Collectors.groupingBy(Pair::getKey)); // Ensure all files in delete list is actually present. This is mandatory for an eventually consistent FS. @@ -597,4 +652,10 @@ public String getBaseFileExtension() { public boolean requireSortedRecords() { return getBaseFileFormat() == HoodieFileFormat.HFILE; } + + public HoodieEngineContext getContext() { + // This is to handle scenarios where this is called at the executor tasks which do not have access + // to engine context, and it ends up being null (as its not serializable and marked transient here). + return context == null ? new HoodieLocalEngineContext(hadoopConfiguration.get()) : context; + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index a6b11af23dc36..5d4a743c39264 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -28,7 +28,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.ReplaceArchivalHelper; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieAvroPayload; @@ -46,6 +46,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.CollectionUtils; @@ -56,6 +57,8 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.metadata.HoodieTableMetadata; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -106,7 +109,7 @@ private Writer openWriter() { } else { return this.writer; } - } catch (InterruptedException | IOException e) { + } catch (IOException e) { throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); } } @@ -195,6 +198,26 @@ private Stream getInstantsToArchive() { .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), HoodieInstant.getComparableAction(i.getAction())))); + // If metadata table is enabled, do not archive instants which are more recent that the latest synced + // instant on the metadata table. This is required for metadata table sync. + if (config.useFileListingMetadata()) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(), + config.getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) { + Option lastSyncedInstantTime = tableMetadata.getSyncedInstantTime(); + + if (lastSyncedInstantTime.isPresent()) { + LOG.info("Limiting archiving of instants to last synced instant on metadata table at " + lastSyncedInstantTime.get()); + instants = instants.filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.LESSER_THAN, + lastSyncedInstantTime.get())); + } else { + LOG.info("Not archiving as there is no instants yet on the metadata table"); + instants = Stream.empty(); + } + } catch (Exception e) { + throw new HoodieException("Error limiting instant archival based on metadata table", e); + } + } + return instants.flatMap(hoodieInstant -> groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream()); @@ -276,10 +299,10 @@ public void archive(HoodieEngineContext context, List instants) t LOG.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { + // TODO HUDI-1518 Cleaner now takes care of removing replaced file groups. This call to deleteReplacedFileGroups can be removed. boolean deleteSuccess = deleteReplacedFileGroups(context, hoodieInstant); if (!deleteSuccess) { - // throw error and stop archival if deleting replaced file groups failed. - throw new HoodieCommitException("Unable to delete file(s) for " + hoodieInstant.getFileName()); + LOG.warn("Unable to delete file(s) for " + hoodieInstant.getFileName() + ", replaced files possibly deleted by cleaner"); } try { deleteAnyLeftOverMarkerFiles(context, hoodieInstant); @@ -335,7 +358,7 @@ private void writeToFile(Schema wrapperSchema, List records) thro Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header); - this.writer = writer.appendBlock(block); + writer.appendBlock(block); records.clear(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java index 8826204cdafe1..9341a1d8038e3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java @@ -18,8 +18,8 @@ package org.apache.hudi.table; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -135,6 +135,7 @@ public Set createdAndMergedDataPaths(HoodieEngineContext context, int pa if (subDirectories.size() > 0) { parallelism = Math.min(subDirectories.size(), parallelism); SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf()); + context.setJobStatus(this.getClass().getSimpleName(), "Obtaining marker files for all created, merged paths"); dataFiles.addAll(context.flatMap(subDirectories, directory -> { Path path = new Path(directory); FileSystem fileSystem = path.getFileSystem(serializedConf.get()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index 96189e7cfa522..6c776cfb077ae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -21,7 +21,7 @@ import java.io.Serializable; import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java index ade2b1885a117..c6f0832445e5e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java @@ -26,8 +26,8 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.collection.Pair; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java index 588437c5149a2..5b37820071a1e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java @@ -21,11 +21,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanFileInfo; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; -import org.apache.hudi.avro.model.HoodieCleanFileInfo; import org.apache.hudi.common.HoodieCleanStat; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -67,12 +67,12 @@ public BaseCleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig co */ HoodieCleanerPlan requestClean(HoodieEngineContext context) { try { - CleanPlanner planner = new CleanPlanner<>(table, config); + CleanPlanner planner = new CleanPlanner<>(context, table, config); Option earliestInstant = planner.getEarliestCommitToRetain(); List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); if (partitionsToClean.isEmpty()) { - LOG.info("Nothing to clean here. It is already clean"); + LOG.info("Nothing to clean here."); return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build(); } LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 405fc81b7a0c8..80727ffafb31b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.CompactionOperation; @@ -30,6 +31,7 @@ import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -76,11 +78,13 @@ public class CleanPlanner implements Ser private final Map fgIdToPendingCompactionOperations; private HoodieTable hoodieTable; private HoodieWriteConfig config; + private transient HoodieEngineContext context; - public CleanPlanner(HoodieTable hoodieTable, HoodieWriteConfig config) { + public CleanPlanner(HoodieEngineContext context, HoodieTable hoodieTable, HoodieWriteConfig config) { + this.context = context; this.hoodieTable = hoodieTable; this.fileSystemView = hoodieTable.getHoodieView(); - this.commitTimeline = hoodieTable.getCompletedCommitTimeline(); + this.commitTimeline = hoodieTable.getCompletedCommitsTimeline(); this.config = config; this.fgIdToPendingCompactionOperations = ((SyncableFileSystemView) hoodieTable.getSliceView()).getPendingCompactionOperations() @@ -112,14 +116,14 @@ public Stream getSavepointedDataFiles(String savepointTime) { /** * Returns list of partitions where clean operations needs to be performed. * - * @param newInstantToRetain New instant to be retained after this cleanup operation + * @param earliestRetainedInstant New instant to be retained after this cleanup operation * @return list of partitions to scan for cleaning * @throws IOException when underlying file-system throws this exception */ - public List getPartitionPathsToClean(Option newInstantToRetain) throws IOException { + public List getPartitionPathsToClean(Option earliestRetainedInstant) throws IOException { switch (config.getCleanerPolicy()) { case KEEP_LATEST_COMMITS: - return getPartitionPathsForCleanByCommits(newInstantToRetain); + return getPartitionPathsForCleanByCommits(earliestRetainedInstant); case KEEP_LATEST_FILE_VERSIONS: return getPartitionPathsForFullCleaning(); default: @@ -161,7 +165,7 @@ private List getPartitionPathsForCleanByCommits(Option in */ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata cleanMetadata, Option newInstantToRetain) { - LOG.warn("Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed " + LOG.info("Incremental Cleaning mode is enabled. Looking up partition-paths that have since changed " + "since last cleaned at " + cleanMetadata.getEarliestCommitToRetain() + ". New Instant to retain : " + newInstantToRetain); return hoodieTable.getCompletedCommitsTimeline().getInstants().filter( @@ -169,10 +173,16 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata cleanMetadata.getEarliestCommitToRetain()) && HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.LESSER_THAN, newInstantToRetain.get().getTimestamp())).flatMap(instant -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), - HoodieCommitMetadata.class); - return commitMetadata.getPartitionToWriteStats().keySet().stream(); + if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) { + HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata.fromBytes( + hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + return Stream.concat(replaceCommitMetadata.getPartitionToReplaceFileIds().keySet().stream(), replaceCommitMetadata.getPartitionToWriteStats().keySet().stream()); + } else { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), + HoodieCommitMetadata.class); + return commitMetadata.getPartitionToWriteStats().keySet().stream(); + } } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -180,14 +190,13 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata } /** - * Scan and list all paritions for cleaning. + * Scan and list all partitions for cleaning. * @return all partitions paths for the dataset. * @throws IOException */ - private List getPartitionPathsForFullCleaning() throws IOException { + private List getPartitionPathsForFullCleaning() { // Go to brute force mode of scanning all partitions - return FSUtils.getAllPartitionPaths(hoodieTable.getMetaClient().getFs(), hoodieTable.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + return FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), config.getBasePath()); } /** @@ -198,13 +207,17 @@ private List getPartitionPathsForFullCleaning() throws IOException { private List getFilesToCleanKeepingLatestVersions(String partitionPath) { LOG.info("Cleaning " + partitionPath + ", retaining latest " + config.getCleanerFileVersionsRetained() + " file versions. "); - List fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList()); List deletePaths = new ArrayList<>(); // Collect all the datafiles savepointed by all the savepoints List savepointedFiles = hoodieTable.getSavepoints().stream() .flatMap(this::getSavepointedDataFiles) .collect(Collectors.toList()); + // In this scenario, we will assume that once replaced a file group automatically becomes eligible for cleaning completely + // In other words, the file versions only apply to the active file groups. + deletePaths.addAll(getReplacedFilesEligibleToClean(savepointedFiles, partitionPath, Option.empty())); + + List fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { int keepVersions = config.getCleanerFileVersionsRetained(); // do not cleanup slice required for pending compaction @@ -228,18 +241,7 @@ private List getFilesToCleanKeepingLatestVersions(String partitio // Delete the remaining files while (fileSliceIterator.hasNext()) { FileSlice nextSlice = fileSliceIterator.next(); - if (nextSlice.getBaseFile().isPresent()) { - HoodieBaseFile dataFile = nextSlice.getBaseFile().get(); - deletePaths.add(new CleanFileInfo(dataFile.getPath(), false)); - if (dataFile.getBootstrapBaseFile().isPresent() && config.shouldCleanBootstrapBaseFile()) { - deletePaths.add(new CleanFileInfo(dataFile.getBootstrapBaseFile().get().getPath(), true)); - } - } - if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { - // If merge on read, then clean the log files for the commits as well - deletePaths.addAll(nextSlice.getLogFiles().map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) - .collect(Collectors.toList())); - } + deletePaths.addAll(getCleanFileInfoForSlice(nextSlice)); } } return deletePaths; @@ -271,7 +273,11 @@ private List getFilesToCleanKeepingLatestCommits(String partition // determine if we have enough commits, to start cleaning. if (commitTimeline.countInstants() > commitsRetained) { - HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get(); + Option earliestCommitToRetainOption = getEarliestCommitToRetain(); + HoodieInstant earliestCommitToRetain = earliestCommitToRetainOption.get(); + // all replaced file groups before earliestCommitToRetain are eligible to clean + deletePaths.addAll(getReplacedFilesEligibleToClean(savepointedFiles, partitionPath, earliestCommitToRetainOption)); + // add active files List fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList()); for (HoodieFileGroup fileGroup : fileGroups) { List fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList()); @@ -324,6 +330,20 @@ private List getFilesToCleanKeepingLatestCommits(String partition } return deletePaths; } + + private List getReplacedFilesEligibleToClean(List savepointedFiles, String partitionPath, Option earliestCommitToRetain) { + final Stream replacedGroups; + if (earliestCommitToRetain.isPresent()) { + replacedGroups = fileSystemView.getReplacedFileGroupsBefore(earliestCommitToRetain.get().getTimestamp(), partitionPath); + } else { + replacedGroups = fileSystemView.getAllReplacedFileGroups(partitionPath); + } + return replacedGroups.flatMap(HoodieFileGroup::getAllFileSlices) + // do not delete savepointed files (archival will make sure corresponding replacecommit file is not deleted) + .filter(slice -> !slice.getBaseFile().isPresent() || !savepointedFiles.contains(slice.getBaseFile().get().getFileName())) + .flatMap(slice -> getCleanFileInfoForSlice(slice).stream()) + .collect(Collectors.toList()); + } /** * Gets the latest version < instantTime. This version file could still be used by queries. @@ -341,6 +361,23 @@ private String getLatestVersionBeforeCommit(List fileSliceList, Hoodi return null; } + private List getCleanFileInfoForSlice(FileSlice nextSlice) { + List cleanPaths = new ArrayList<>(); + if (nextSlice.getBaseFile().isPresent()) { + HoodieBaseFile dataFile = nextSlice.getBaseFile().get(); + cleanPaths.add(new CleanFileInfo(dataFile.getPath(), false)); + if (dataFile.getBootstrapBaseFile().isPresent() && config.shouldCleanBootstrapBaseFile()) { + cleanPaths.add(new CleanFileInfo(dataFile.getBootstrapBaseFile().get().getPath(), true)); + } + } + if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { + // If merge on read, then clean the log files for the commits as well + cleanPaths.addAll(nextSlice.getLogFiles().map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) + .collect(Collectors.toList())); + } + return cleanPaths; + } + /** * Returns files to be cleaned for the given partitionPath based on cleaning policy. */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java new file mode 100644 index 0000000000000..97407e3464d79 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java @@ -0,0 +1,74 @@ +/* + * 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.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +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.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +public abstract class BaseClusteringPlanActionExecutor extends BaseActionExecutor> { + + private final Option> extraMetadata; + + public BaseClusteringPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + } + + protected abstract Option createClusteringPlan(); + + @Override + public Option execute() { + Option planOption = createClusteringPlan(); + if (planOption.isPresent()) { + HoodieInstant clusteringInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); + try { + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setOperationType(WriteOperationType.CLUSTER.name()) + .setExtraMetadata(extraMetadata.orElse(Collections.emptyMap())) + .setClusteringPlan(planOption.get()) + .build(); + table.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, + TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + } catch (IOException ioe) { + throw new HoodieIOException("Exception scheduling clustering", ioe); + } + } + return planOption; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java new file mode 100644 index 0000000000000..dce6eeac3bd0b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.avro.Schema; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; +import java.util.Map; + +/** + * Pluggable implementation for writing data into new file groups based on ClusteringPlan. + */ +public abstract class ClusteringExecutionStrategy implements Serializable { + private static final Logger LOG = LogManager.getLogger(ClusteringExecutionStrategy.class); + + private final HoodieTable hoodieTable; + private final HoodieEngineContext engineContext; + private final HoodieWriteConfig writeConfig; + + public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + this.hoodieTable = table; + this.engineContext = engineContext; + } + + /** + * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. The number of new + * file groups created is bounded by numOutputGroups. + * Note that commit is not done as part of strategy. commit is callers responsibility. + */ + public abstract O performClustering(final I inputRecords, final int numOutputGroups, final String instantTime, + final Map strategyParams, final Schema schema); + + protected HoodieTable getHoodieTable() { + return this.hoodieTable; + } + + protected HoodieEngineContext getEngineContext() { + return this.engineContext; + } + + protected HoodieWriteConfig getWriteConfig() { + return this.writeConfig; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java new file mode 100644 index 0000000000000..273ebce761e26 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -0,0 +1,142 @@ +/* + * 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.cluster.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.client.utils.FileSliceMetricUtils; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Pluggable implementation for scheduling clustering and creating ClusteringPlan. + */ +public abstract class ClusteringPlanStrategy implements Serializable { + private static final Logger LOG = LogManager.getLogger(ClusteringPlanStrategy.class); + + public static final int CLUSTERING_PLAN_VERSION_1 = 1; + + private final HoodieTable hoodieTable; + private final transient HoodieEngineContext engineContext; + private final HoodieWriteConfig writeConfig; + + public ClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + this.hoodieTable = table; + this.engineContext = engineContext; + } + + /** + * Generate metadata for grouping eligible files and create a plan. Note that data is not moved around + * as part of this step. + * + * If there is no data available to cluster, return None. + */ + public abstract Option generateClusteringPlan(); + + /** + * Return file slices eligible for clustering. FileIds in pending clustering/compaction are not eligible for clustering. + */ + protected Stream getFileSlicesEligibleForClustering(String partition) { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView(); + Set fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet())); + + return hoodieTable.getSliceView().getLatestFileSlices(partition) + // file ids already in clustering are not eligible + .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())); + } + + /** + * Get parameters specific to strategy. These parameters are passed from 'schedule clustering' step to + * 'execute clustering' step. 'execute clustering' step is typically async. So these params help with passing any required + * context from schedule to run step. + */ + protected abstract Map getStrategyParams(); + + /** + * Returns any specific parameters to be stored as part of clustering metadata. + */ + protected Map getExtraMetadata() { + return Collections.emptyMap(); + } + + /** + * Version to support future changes for plan. + */ + protected int getPlanVersion() { + return CLUSTERING_PLAN_VERSION_1; + } + + /** + * Transform {@link FileSlice} to {@link HoodieSliceInfo}. + */ + protected static List getFileSliceInfo(List slices) { + return slices.stream().map(slice -> new HoodieSliceInfo().newBuilder() + .setPartitionPath(slice.getPartitionPath()) + .setFileId(slice.getFileId()) + .setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(StringUtils.EMPTY_STRING)) + .setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toList())) + .setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(StringUtils.EMPTY_STRING)).orElse(StringUtils.EMPTY_STRING)) + .build()).collect(Collectors.toList()); + } + + /** + * Generate metrics for the data to be clustered. + */ + protected Map buildMetrics(List fileSlices) { + Map metrics = new HashMap<>(); + FileSliceMetricUtils.addFileSliceCommonMetrics(fileSlices, metrics, getWriteConfig().getParquetMaxFileSize()); + return metrics; + } + + protected HoodieTable getHoodieTable() { + return this.hoodieTable; + } + + protected HoodieEngineContext getEngineContext() { + return this.engineContext; + } + + protected HoodieWriteConfig getWriteConfig() { + return this.writeConfig; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java new file mode 100644 index 0000000000000..d3b39f59eda0c --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -0,0 +1,102 @@ +/* + * 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.cluster.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieClusteringStrategy; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Scheduling strategy with restriction that clustering groups can only contain files from same partition. + */ +public abstract class PartitionAwareClusteringPlanStrategy extends ClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(PartitionAwareClusteringPlanStrategy.class); + + public PartitionAwareClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + /** + * Create Clustering group based on files eligible for clustering in the partition. + */ + protected abstract Stream buildClusteringGroupsForPartition(String partitionPath, + List fileSlices); + + /** + * Return list of partition paths to be considered for clustering. + */ + protected List filterPartitionPaths(List partitionPaths) { + return partitionPaths; + } + + @Override + public Option generateClusteringPlan() { + HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); + LOG.info("Scheduling clustering for " + metaClient.getBasePath()); + HoodieWriteConfig config = getWriteConfig(); + List partitionPaths = FSUtils.getAllPartitionPaths(getEngineContext(), config.getMetadataConfig(), metaClient.getBasePath()); + + // filter the partition paths if needed to reduce list status + partitionPaths = filterPartitionPaths(partitionPaths); + + if (partitionPaths.isEmpty()) { + // In case no partitions could be picked, return no clustering plan + return Option.empty(); + } + + List clusteringGroups = getEngineContext().flatMap(partitionPaths, + partitionPath -> { + List fileSlicesEligible = getFileSlicesEligibleForClustering(partitionPath).collect(Collectors.toList()); + return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible).limit(getWriteConfig().getClusteringMaxNumGroups()); + }, + partitionPaths.size()) + .stream().limit(getWriteConfig().getClusteringMaxNumGroups()).collect(Collectors.toList()); + + if (clusteringGroups.isEmpty()) { + LOG.info("No data available to cluster"); + return Option.empty(); + } + + HoodieClusteringStrategy strategy = HoodieClusteringStrategy.newBuilder() + .setStrategyClassName(getWriteConfig().getClusteringExecutionStrategyClass()) + .setStrategyParams(getStrategyParams()) + .build(); + + return Option.of(HoodieClusteringPlan.newBuilder() + .setStrategy(strategy) + .setInputGroups(clusteringGroups) + .setExtraMetadata(getExtraMetadata()) + .setVersion(getPlanVersion()) + .build()); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java new file mode 100644 index 0000000000000..3ce4f04e3d035 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; + +import java.util.Set; + +/** + * When file groups in clustering, write records to these file group need to check. + */ +public abstract class UpdateStrategy, I> { + + protected final HoodieEngineContext engineContext; + protected Set fileGroupsInPendingClustering; + + protected UpdateStrategy(HoodieEngineContext engineContext, Set fileGroupsInPendingClustering) { + this.engineContext = engineContext; + this.fileGroupsInPendingClustering = fileGroupsInPendingClustering; + } + + /** + * Check the update records to the file group in clustering. + * @param taggedRecordsRDD the records to write, tagged with target file id, + * future can update tagged records location to a different fileId. + * @return the recordsRDD strategy updated + */ + public abstract I handleUpdate(I taggedRecordsRDD); + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java index 3ead7a07d9698..d2aa8627bd275 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java @@ -27,8 +27,21 @@ public abstract class AbstractBulkInsertHelper { + /** + * Mark instant as inflight, write input records, update index and return result. + */ public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, BaseCommitActionExecutor executor, boolean performDedupe, Option> userDefinedBulkInsertPartitioner); + + /** + * Only write input records. Does not change timeline/index. Return information about new files created. + */ + public abstract O bulkInsert(I inputRecords, String instantTime, + HoodieTable table, HoodieWriteConfig config, + boolean performDedupe, + Option> userDefinedBulkInsertPartitioner, + boolean addMetadataFields, + int parallelism); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java index c2d2df7ebac99..ac0f2596f490e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java index caa6ecdb953a7..455952ae5f7d7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 71de9b6fc6f73..4a8ddf5c57997 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java similarity index 100% rename from hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/Partitioner.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java index d21c7d96d113a..9e09b8c82761c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index e5212e8f43a5b..c92c0b3a0237e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java index 6c631c462bda1..ff2dfbd4c1ecb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java @@ -20,14 +20,13 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.util.CompactionUtils; -import org.apache.hudi.common.util.Option; +import org.apache.hudi.client.utils.FileSliceMetricUtils; import org.apache.hudi.config.HoodieWriteConfig; import java.io.Serializable; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -50,32 +49,14 @@ public abstract class CompactionStrategy implements Serializable { * Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the metrics they need * to decide on the priority. * - * @param dataFile - Base file to compact - * @param partitionPath - Partition path - * @param logFiles - List of log files to compact with the base file + * @param writeConfig write configuration. + * @param slice fileSlice to capture metrics for. * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieWriteConfig writeConfig, Option dataFile, - String partitionPath, List logFiles) { + public Map captureMetrics(HoodieWriteConfig writeConfig, FileSlice slice) { Map metrics = new HashMap<>(); long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); - // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) - .reduce(Long::sum).orElse(0L); - // Total read will be the base file + all the log files - Long totalIORead = - FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L) + totalLogFileSize); - // Total write will be similar to the size of the base file - Long totalIOWrite = - FSUtils.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize); - // Total IO will the the IO for read + write - long totalIO = totalIORead + totalIOWrite; - // Save these metrics and we will use during the filter - metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue()); - metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue()); - metrics.put(TOTAL_IO_MB, (double) totalIO); - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); - metrics.put(TOTAL_LOG_FILES, (double) logFiles.size()); + FileSliceMetricUtils.addFileSliceCommonMetrics(Collections.singletonList(slice), metrics, defaultMaxParquetFileSize); return metrics; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java index c9a811a1cceeb..fe4b474593886 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -20,14 +20,10 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; /** @@ -40,21 +36,6 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements Comparator { - private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; - - @Override - public Map captureMetrics(HoodieWriteConfig config, Option dataFile, - String partitionPath, List logFiles) { - Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); - - // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) - .reduce(Long::sum).orElse(0L); - // save the metrics needed during the order - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); - return metrics; - } - @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 0bbc0a05f9e59..0dd2a7cfb4df1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java index 40526b86f2cdb..cc596ba3422b7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java @@ -18,9 +18,10 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -37,6 +38,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.Objects; /** * Performs rollback using marker files generated during the write.. @@ -51,9 +53,9 @@ public abstract class AbstractMarkerBasedRollbackStrategy table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) { this.table = table; @@ -88,6 +90,7 @@ protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend); String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName()); String partitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), new Path(basePath, appendBaseFilePath).getParent()); + final Map writtenLogFileSizeMap = getWrittenLogFileSizeMap(partitionPath, baseCommitTime, fileId); HoodieLogFormat.Writer writer = null; try { @@ -108,7 +111,7 @@ protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant // generate metadata Map header = RollbackUtils.generateHeader(instantToRollback.getTimestamp(), instantTime); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); + writer.appendBlock(new HoodieCommandBlock(header)); } finally { try { if (writer != null) { @@ -119,10 +122,26 @@ protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant } } + // the information of files appended to is required for metadata sync + Map filesToNumBlocksRollback = Collections.singletonMap( + table.getMetaClient().getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()), + 1L); + return HoodieRollbackStat.newBuilder() .withPartitionPath(partitionPath) - // we don't use this field per se. Avoiding the extra file status call. - .withRollbackBlockAppendResults(Collections.emptyMap()) - .build(); + .withRollbackBlockAppendResults(filesToNumBlocksRollback) + .withWrittenLogFileSizeMap(writtenLogFileSizeMap).build(); + } + + /** + * Returns written log file size map for the respective baseCommitTime to assist in metadata table syncing. + * @param partitionPath partition path of interest + * @param baseCommitTime base commit time of interest + * @param fileId fileId of interest + * @return Map + * @throws IOException + */ + protected Map getWrittenLogFileSizeMap(String partitionPath, String baseCommitTime, String fileId) throws IOException { + return Collections.EMPTY_MAP; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java index c60cb7f5e7932..fa74f7f6e86bc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java index 0e747a58dc673..2e751443abc00 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.HoodieTimer; @@ -28,7 +28,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -57,7 +56,7 @@ public BaseMergeOnReadRollbackActionExecutor(HoodieEngineContext context, } @Override - protected List executeRollback() throws IOException { + protected List executeRollback() { HoodieTimer rollbackTimer = new HoodieTimer(); rollbackTimer.startTimer(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 40206ca45e1c3..3a579535528cc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -19,9 +19,9 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index 897b448fb5340..bfbe577f9429f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -20,11 +20,11 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -74,31 +74,30 @@ static HoodieRollbackStat mergeRollbackStat(HoodieRollbackStat stat1, HoodieRoll final List successDeleteFiles = new ArrayList<>(); final List failedDeleteFiles = new ArrayList<>(); final Map commandBlocksCount = new HashMap<>(); - final List filesToRollback = new ArrayList<>(); + final Map writtenLogFileSizeMap = new HashMap<>(); Option.ofNullable(stat1.getSuccessDeleteFiles()).ifPresent(successDeleteFiles::addAll); Option.ofNullable(stat2.getSuccessDeleteFiles()).ifPresent(successDeleteFiles::addAll); Option.ofNullable(stat1.getFailedDeleteFiles()).ifPresent(failedDeleteFiles::addAll); Option.ofNullable(stat2.getFailedDeleteFiles()).ifPresent(failedDeleteFiles::addAll); Option.ofNullable(stat1.getCommandBlocksCount()).ifPresent(commandBlocksCount::putAll); Option.ofNullable(stat2.getCommandBlocksCount()).ifPresent(commandBlocksCount::putAll); - return new HoodieRollbackStat(stat1.getPartitionPath(), successDeleteFiles, failedDeleteFiles, commandBlocksCount); + Option.ofNullable(stat1.getWrittenLogFileSizeMap()).ifPresent(writtenLogFileSizeMap::putAll); + Option.ofNullable(stat2.getWrittenLogFileSizeMap()).ifPresent(writtenLogFileSizeMap::putAll); + return new HoodieRollbackStat(stat1.getPartitionPath(), successDeleteFiles, failedDeleteFiles, commandBlocksCount, writtenLogFileSizeMap); } /** * Generate all rollback requests that needs rolling back this action without actually performing rollback for COW table type. * @param fs instance of {@link FileSystem} to use. * @param basePath base path of interest. - * @param shouldAssumeDatePartitioning {@code true} if date partitioning should be assumed. {@code false} otherwise. + * @param config instance of {@link HoodieWriteConfig} to use. * @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected. */ - public static List generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, boolean shouldAssumeDatePartitioning) { - try { - return FSUtils.getAllPartitionPaths(fs, basePath, shouldAssumeDatePartitioning).stream() - .map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction) - .collect(Collectors.toList()); - } catch (IOException e) { - throw new HoodieIOException("Error generating rollback requests", e); - } + public static List generateRollbackRequestsByListingCOW(HoodieEngineContext engineContext, + String basePath, HoodieWriteConfig config) { + return FSUtils.getAllPartitionPaths(engineContext, config.getMetadataConfig(), basePath).stream() + .map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction) + .collect(Collectors.toList()); } /** @@ -112,8 +111,7 @@ public static List generateRollbackRequestsByListin public static List generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException { String commit = instantToRollback.getTimestamp(); HoodieWriteConfig config = table.getConfig(); - List partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1); context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests"); return context.flatMap(partitions, partitionPath -> { @@ -121,6 +119,7 @@ public static List generateRollbackRequestsUsingFil List partitionRollbackRequests = new ArrayList<>(); switch (instantToRollback.getAction()) { case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.REPLACE_COMMIT_ACTION: LOG.info("Rolling back commit action."); partitionRollbackRequests.add( ListingBasedRollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index 16fd9a481e02d..de1d973a2fa2c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -20,7 +20,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -31,13 +31,14 @@ import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import scala.Tuple2; import java.io.IOException; import java.util.List; @@ -89,15 +90,15 @@ public HoodieSavepointMetadata execute() { "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained); context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime); - Map> latestFilesMap = context.mapToPair(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), - table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()), partitionPath -> { - // Scan all partitions files with this commit time - LOG.info("Collecting latest files in partition path " + partitionPath); - TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); - List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) - .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); - return new Tuple2<>(partitionPath, latestFiles); - }, null); + List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + Map> latestFilesMap = context.mapToPair(partitions, partitionPath -> { + // Scan all partitions files with this commit time + LOG.info("Collecting latest files in partition path " + partitionPath); + TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); + List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) + .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); + return new ImmutablePair<>(partitionPath, latestFiles); + }, null); HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint table.getActiveTimeline().createNewInstant( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java index cafb8167b6fc5..7476a1630e247 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java index 27389d923bd6a..2b525841e2217 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java index fdf1261b31201..2ba688f229373 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; /** diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyTaskContextSupplier.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyTaskContextSupplier.java index 022170567fa35..d2c07e35509c1 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyTaskContextSupplier.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyTaskContextSupplier.java @@ -18,8 +18,8 @@ package org.apache.hudi; -import org.apache.hudi.client.common.EngineProperty; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.util.Option; import java.util.function.Supplier; diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java index 360fa054b8def..1f6a2a708c8c8 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java @@ -18,8 +18,10 @@ package org.apache.hudi.config; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.config.HoodieWriteConfig.Builder; +import org.apache.hudi.index.HoodieIndex; import org.junit.jupiter.api.Test; import java.io.ByteArrayInputStream; @@ -54,6 +56,21 @@ public void testPropertyLoading() throws IOException { assertEquals(2, config.getMinCommitsToKeep()); } + @Test + public void testDefaultIndexAccordingToEngineType() { + // default bloom + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build(); + assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType()); + + // spark default bloom + writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.SPARK).withPath("/tmp").build(); + assertEquals(HoodieIndex.IndexType.BLOOM, writeConfig.getIndexType()); + + // flink default in-memory + writeConfig = HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath("/tmp").build(); + assertEquals(HoodieIndex.IndexType.INMEMORY, writeConfig.getIndexType()); + } + private ByteArrayOutputStream saveParamsIntoOutputStream(Map params) throws IOException { Properties properties = new Properties(); properties.putAll(params); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java similarity index 100% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/Assertions.java diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java similarity index 71% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java index e167a0f4b6650..694d44c10e977 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java @@ -21,10 +21,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.HoodieAvroWriteSupport; -import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.bloom.BloomFilterFactory; -import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -37,7 +35,6 @@ import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieParquetWriter; -import org.apache.hudi.table.HoodieTable; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -57,7 +54,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.stream.Collectors; import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; @@ -65,35 +61,15 @@ public class HoodieWriteableTestTable extends HoodieTestTable { private static final Logger LOG = LogManager.getLogger(HoodieWriteableTestTable.class); - private final Schema schema; - private final BloomFilter filter; + protected final Schema schema; + protected final BloomFilter filter; - private HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { + protected HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { super(basePath, fs, metaClient); this.schema = schema; this.filter = filter; } - public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { - return new HoodieWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter); - } - - public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { - BloomFilter filter = BloomFilterFactory - .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); - return of(metaClient, schema, filter); - } - - public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - return of(metaClient, schema); - } - - public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) { - HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - return of(metaClient, schema, filter); - } - @Override public HoodieWriteableTestTable addCommit(String instantTime) throws Exception { return (HoodieWriteableTestTable) super.addCommit(instantTime); @@ -104,29 +80,7 @@ public HoodieWriteableTestTable forCommit(String instantTime) { return (HoodieWriteableTestTable) super.forCommit(instantTime); } - public String getFileIdWithInserts(String partition) throws Exception { - return getFileIdWithInserts(partition, new HoodieRecord[0]); - } - - public String getFileIdWithInserts(String partition, HoodieRecord... records) throws Exception { - return getFileIdWithInserts(partition, Arrays.asList(records)); - } - - public String getFileIdWithInserts(String partition, List records) throws Exception { - String fileId = UUID.randomUUID().toString(); - withInserts(partition, fileId, records); - return fileId; - } - - public HoodieWriteableTestTable withInserts(String partition, String fileId) throws Exception { - return withInserts(partition, fileId, new HoodieRecord[0]); - } - - public HoodieWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception { - return withInserts(partition, fileId, Arrays.asList(records)); - } - - public HoodieWriteableTestTable withInserts(String partition, String fileId, List records) throws Exception { + public HoodieWriteableTestTable withInserts(String partition, String fileId, List records, TaskContextSupplier contextSupplier) throws Exception { FileCreateUtils.createPartitionMetaFile(basePath, partition); String fileName = baseFileName(currentInstantTime, fileId); @@ -138,7 +92,7 @@ public HoodieWriteableTestTable withInserts(String partition, String fileId, Lis try (HoodieParquetWriter writer = new HoodieParquetWriter( currentInstantTime, new Path(Paths.get(basePath, partition, fileName).toString()), - config, schema, new SparkTaskContextSupplier())) { + config, schema, contextSupplier)) { int seqId = 1; for (HoodieRecord record : records) { GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java similarity index 100% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java index e876c08bc53f1..3faa4b2c32b2f 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java @@ -19,7 +19,7 @@ package org.apache.hudi.testutils.providers; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; public interface HoodieEngineContextProvider { HoodieEngineContext context(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java new file mode 100644 index 0000000000000..af4c4fbfc0cb4 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java @@ -0,0 +1,65 @@ +/* + * 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.utils; + +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestConcatenatingIterator { + + // Simple test for iterator concatenation + @Test + public void testConcatBasic() { + Iterator i1 = Arrays.asList(5, 3, 2, 1).iterator(); + Iterator i2 = new ArrayList().iterator(); // empty iterator + Iterator i3 = Arrays.asList(3).iterator(); + + ConcatenatingIterator ci = new ConcatenatingIterator<>(Arrays.asList(i1, i2, i3)); + List allElements = new ArrayList<>(); + while (ci.hasNext()) { + allElements.add(ci.next()); + } + + assertEquals(5, allElements.size()); + assertEquals(Arrays.asList(5, 3, 2, 1, 3), allElements); + } + + @Test + public void testConcatError() { + Iterator i1 = new ArrayList().iterator(); // empty iterator + + ConcatenatingIterator ci = new ConcatenatingIterator<>(Arrays.asList(i1)); + assertFalse(ci.hasNext()); + try { + ci.next(); + fail("expected error for empty iterator"); + } catch (IllegalArgumentException e) { + // + } + } +} \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index e07f0c672262e..7408b4a907ca4 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -15,17 +15,16 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi-client org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 hudi-flink-client - ${parent.version} + 0.8.0-SNAPSHOT hudi-flink-client jar @@ -115,6 +114,28 @@ test + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + test + tests + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + test + tests + + org.junit.jupiter diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java index bb405e21f961f..aab248fc3cf16 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/FlinkTaskContextSupplier.java @@ -18,8 +18,8 @@ package org.apache.hudi.client; -import org.apache.hudi.client.common.EngineProperty; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.util.Option; import org.apache.flink.api.common.functions.RuntimeContext; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index f975406e4505b..d845b90750238 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -18,9 +18,9 @@ package org.apache.hudi.client; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -106,8 +106,7 @@ public List upsert(List> records, String instantTim HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); - setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.UPSERT); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -125,8 +124,7 @@ public List insert(List> records, String instantTim HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateUpsertSchema(); - setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.INSERT); HoodieWriteMetadata> result = table.insert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -158,7 +156,7 @@ public List bulkInsertPreppedRecords(List> preppedR public List delete(List keys, String instantTime) { HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - setOperationType(WriteOperationType.DELETE); + preWrite(instantTime, WriteOperationType.DELETE); HoodieWriteMetadata> result = table.delete(context,instantTime, keys); return postWrite(result, instantTime, table); } @@ -188,6 +186,11 @@ protected List compact(String compactionInstantTime, boolean should throw new HoodieNotSupportedException("Compaction is not supported yet"); } + @Override + public HoodieWriteMetadata> cluster(final String clusteringInstant, final boolean shouldComplete) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + @Override protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java index 74c921fd0cb8b..6c7f44b57ea53 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/common/HoodieFlinkEngineContext.java @@ -19,10 +19,13 @@ package org.apache.hudi.client.common; import org.apache.hudi.client.FlinkTaskContextSupplier; -import org.apache.hudi.client.common.function.SerializableConsumer; -import org.apache.hudi.client.common.function.SerializableFunction; -import org.apache.hudi.client.common.function.SerializablePairFunction; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.function.SerializableConsumer; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; import org.apache.flink.api.common.functions.RuntimeContext; @@ -33,12 +36,12 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import scala.Tuple2; +import org.apache.hudi.common.util.collection.Pair; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingFlatMapWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingForeachWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingMapToPairWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingMapWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; /** * A flink engine implementation of HoodieEngineContext. @@ -76,7 +79,7 @@ public void foreach(List data, SerializableConsumer consumer, int para @Override public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { - return data.stream().map(throwingMapToPairWrapper(func)).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2)); + return data.stream().map(throwingMapToPairWrapper(func)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); } @Override diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java index f0e51220d7c98..8769f63e37258 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -19,7 +19,7 @@ package org.apache.hudi.execution; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java index 427212c6f897b..cfb21d9da43b8 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/FlinkHoodieIndex.java @@ -20,8 +20,8 @@ import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -29,6 +29,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.index.bloom.FlinkHoodieBloomIndex; import org.apache.hudi.index.state.FlinkInMemoryStateIndex; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.table.HoodieTable; @@ -58,6 +59,8 @@ public static FlinkHoodieIndex createIndex(HoodieFlinkEngineContext context, Hoo switch (config.getIndexType()) { case INMEMORY: return new FlinkInMemoryStateIndex<>(context, config); + case BLOOM: + return new FlinkHoodieBloomIndex(config); default: throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java new file mode 100644 index 0000000000000..6a3edc7c38bea --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/FlinkHoodieBloomIndex.java @@ -0,0 +1,267 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.MetadataNotFoundException; +import org.apache.hudi.index.FlinkHoodieIndex; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.io.HoodieRangeInfoHandle; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import com.beust.jcommander.internal.Lists; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import scala.Tuple2; + +import static java.util.stream.Collectors.mapping; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions; + +/** + * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata. + */ +@SuppressWarnings("checkstyle:LineLength") +public class FlinkHoodieBloomIndex extends FlinkHoodieIndex { + + private static final Logger LOG = LogManager.getLogger(FlinkHoodieBloomIndex.class); + + public FlinkHoodieBloomIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + public List> tagLocation(List> records, HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) { + // Step 1: Extract out thinner Map of (partitionPath, recordKey) + Map> partitionRecordKeyMap = new HashMap<>(); + records.forEach(record -> { + if (partitionRecordKeyMap.containsKey(record.getPartitionPath())) { + partitionRecordKeyMap.get(record.getPartitionPath()).add(record.getRecordKey()); + } else { + List recordKeys = Lists.newArrayList(); + recordKeys.add(record.getRecordKey()); + partitionRecordKeyMap.put(record.getPartitionPath(), recordKeys); + } + }); + + // Step 2: Lookup indexes for all the partition/recordkey pair + Map keyFilenamePairMap = + lookupIndex(partitionRecordKeyMap, context, hoodieTable); + + if (LOG.isDebugEnabled()) { + long totalTaggedRecords = keyFilenamePairMap.values().size(); + LOG.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords); + } + + // Step 3: Tag the incoming records, as inserts or updates, by joining with existing record keys + List> taggedRecords = tagLocationBacktoRecords(keyFilenamePairMap, records); + + return taggedRecords; + } + + /** + * Lookup the location for each record key and return the pair for all record keys already + * present and drop the record keys if not present. + */ + private Map lookupIndex( + Map> partitionRecordKeyMap, final HoodieEngineContext context, + final HoodieTable hoodieTable) { + // Obtain records per partition, in the incoming records + Map recordsPerPartition = new HashMap<>(); + partitionRecordKeyMap.keySet().forEach(k -> recordsPerPartition.put(k, Long.valueOf(partitionRecordKeyMap.get(k).size()))); + List affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet()); + + // Step 2: Load all involved files as pairs + List> fileInfoList = + loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable); + final Map> partitionToFileInfo = + fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList()))); + + // Step 3: Obtain a List, for each incoming record, that already exists, with the file id, + // that contains it. + List> fileComparisons = + explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyMap); + return findMatchingFilesForRecordKeys(fileComparisons, hoodieTable); + } + + /** + * Load all involved files as pair List. + */ + //TODO duplicate code with spark, we can optimize this method later + List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, + final HoodieTable hoodieTable) { + // Obtain the latest data files from all the partitions. + List> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() + .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) + .collect(toList()); + + if (config.getBloomIndexPruneByRanges()) { + // also obtain file ranges, if range pruning is enabled + context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); + return context.map(partitionPathFileIDList, pf -> { + try { + HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); + String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys(); + return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1])); + } catch (MetadataNotFoundException me) { + LOG.warn("Unable to find range metadata in file :" + pf); + return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())); + } + }, Math.max(partitionPathFileIDList.size(), 1)); + } else { + return partitionPathFileIDList.stream() + .map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList()); + } + } + + @Override + public boolean rollbackCommit(String instantTime) { + // Nope, don't need to do anything. + return true; + } + + /** + * This is not global, since we depend on the partitionPath to do the lookup. + */ + @Override + public boolean isGlobal() { + return false; + } + + /** + * No indexes into log files yet. + */ + @Override + public boolean canIndexLogFiles() { + return false; + } + + /** + * Bloom filters are stored, into the same data files. + */ + @Override + public boolean isImplicitWithStorage() { + return true; + } + + /** + * For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be + * checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files + * to be compared gets cut down a lot from range pruning. + *

+ * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on + * recordKey ranges in the index info. + */ + List> explodeRecordsWithFileComparisons( + final Map> partitionToFileIndexInfo, + Map> partitionRecordKeyMap) { + IndexFileFilter indexFileFilter = + config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo) + : new ListBasedIndexFileFilter(partitionToFileIndexInfo); + + List> fileRecordPairs = new ArrayList<>(); + partitionRecordKeyMap.keySet().forEach(partitionPath -> { + List hoodieRecordKeys = partitionRecordKeyMap.get(partitionPath); + hoodieRecordKeys.forEach(hoodieRecordKey -> { + indexFileFilter.getMatchingFilesAndPartition(partitionPath, hoodieRecordKey).forEach(partitionFileIdPair -> { + fileRecordPairs.add(new Tuple2<>(partitionFileIdPair.getRight(), + new HoodieKey(hoodieRecordKey, partitionPath))); + }); + }); + }); + return fileRecordPairs; + } + + /** + * Find out pair. + */ + Map findMatchingFilesForRecordKeys( + List> fileComparisons, + HoodieTable hoodieTable) { + + fileComparisons = fileComparisons.stream().sorted((o1, o2) -> o1._1.compareTo(o2._1)).collect(toList()); + + List keyLookupResults = new ArrayList<>(); + + Iterator> iterator = new HoodieFlinkBloomIndexCheckFunction(hoodieTable, config).apply(fileComparisons.iterator()); + while (iterator.hasNext()) { + keyLookupResults.addAll(iterator.next()); + } + + Map hoodieRecordLocationMap = new HashMap<>(); + + keyLookupResults = keyLookupResults.stream().filter(lr -> lr.getMatchingRecordKeys().size() > 0).collect(toList()); + keyLookupResults.forEach(lookupResult -> { + lookupResult.getMatchingRecordKeys().forEach(r -> { + hoodieRecordLocationMap.put(new HoodieKey(r, lookupResult.getPartitionPath()), new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())); + }); + }); + + return hoodieRecordLocationMap; + } + + + /** + * Tag the back to the original HoodieRecord List. + */ + protected List> tagLocationBacktoRecords( + Map keyFilenamePair, List> records) { + Map> keyRecordPairMap = new HashMap<>(); + records.forEach(r -> keyRecordPairMap.put(r.getKey(), r)); + // Here as the record might have more data than rowKey (some rowKeys' fileId is null), + // so we do left outer join. + List, HoodieRecordLocation>> newList = new ArrayList<>(); + keyRecordPairMap.keySet().forEach(k -> { + if (keyFilenamePair.containsKey(k)) { + newList.add(new Tuple2(keyRecordPairMap.get(k), keyFilenamePair.get(k))); + } else { + newList.add(new Tuple2(keyRecordPairMap.get(k), null)); + } + }); + List> res = Lists.newArrayList(); + for (Tuple2, HoodieRecordLocation> v : newList) { + res.add(HoodieIndexUtils.getTaggedRecord(v._1, Option.ofNullable(v._2))); + } + return res; + } + + @Override + public List updateLocation(List writeStatusList, HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) { + return writeStatusList; + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/HoodieFlinkBloomIndexCheckFunction.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/HoodieFlinkBloomIndexCheckFunction.java new file mode 100644 index 0000000000000..33ec9e65d0b73 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/bloom/HoodieFlinkBloomIndexCheckFunction.java @@ -0,0 +1,127 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult; +import org.apache.hudi.table.HoodieTable; + +import java.util.function.Function; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import scala.Tuple2; + +/** + * Function performing actual checking of list containing (fileId, hoodieKeys) against the actual files. + */ +//TODO we can move this class into the hudi-client-common and reuse it for spark client +public class HoodieFlinkBloomIndexCheckFunction + implements Function>, Iterator>> { + + private final HoodieTable hoodieTable; + + private final HoodieWriteConfig config; + + public HoodieFlinkBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) { + this.hoodieTable = hoodieTable; + this.config = config; + } + + @Override + public Iterator> apply(Iterator> fileParitionRecordKeyTripletItr) { + return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr); + } + + @Override + public Function>> compose(Function>> before) { + return null; + } + + @Override + public Function>, V> andThen(Function>, ? extends V> after) { + return null; + } + + class LazyKeyCheckIterator extends LazyIterableIterator, List> { + + private HoodieKeyLookupHandle keyLookupHandle; + + LazyKeyCheckIterator(Iterator> filePartitionRecordKeyTripletItr) { + super(filePartitionRecordKeyTripletItr); + } + + @Override + protected void start() { + } + + @Override + protected List computeNext() { + List ret = new ArrayList<>(); + try { + // process one file in each go. + while (inputItr.hasNext()) { + Tuple2 currentTuple = inputItr.next(); + String fileId = currentTuple._1; + String partitionPath = currentTuple._2.getPartitionPath(); + String recordKey = currentTuple._2.getRecordKey(); + Pair partitionPathFilePair = Pair.of(partitionPath, fileId); + + // lazily init state + if (keyLookupHandle == null) { + keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair); + } + + // if continue on current file + if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) { + keyLookupHandle.addKey(recordKey); + } else { + // do the actual checking of file & break out + ret.add(keyLookupHandle.getLookupResult()); + keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair); + keyLookupHandle.addKey(recordKey); + break; + } + } + + // handle case, where we ran out of input, close pending work, update return val + if (!inputItr.hasNext()) { + ret.add(keyLookupHandle.getLookupResult()); + } + } catch (Throwable e) { + if (e instanceof HoodieException) { + throw e; + } + throw new HoodieIndexException("Error checking bloom filter index. ", e); + } + return ret; + } + + @Override + protected void end() { + } + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java index d3fdf67d76a92..4354ea3f9f379 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/index/state/FlinkInMemoryStateIndex.java @@ -19,8 +19,8 @@ package org.apache.hudi.index.state; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index acb010c313065..4d7edd79d7baa 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -19,12 +19,13 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -83,6 +84,11 @@ public HoodieWriteMetadata> delete(HoodieEngineContext context return new FlinkDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute(); } + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + throw new HoodieNotSupportedException("DeletePartitions is not supported yet"); + } + @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { return new FlinkUpsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute(); @@ -106,6 +112,11 @@ public HoodieWriteMetadata> insertOverwrite(HoodieEngineContex throw new HoodieNotSupportedException("InsertOverWrite is not supported yet"); } + @Override + public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, List> records) { + throw new HoodieNotSupportedException("insertOverwriteTable is not supported yet"); + } + @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); @@ -116,6 +127,16 @@ public HoodieWriteMetadata> compact(HoodieEngineContext contex throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } + @Override + public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { + throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + } + + @Override + public HoodieWriteMetadata> cluster(final HoodieEngineContext context, final String clusteringInstantTime) { + throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + } + @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { throw new HoodieNotSupportedException("Bootstrap is not supported yet"); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java index 51ce54a42da4f..994a49d22313c 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkMergeOnReadTable.java @@ -18,7 +18,7 @@ package org.apache.hudi.table; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 3c09b38e64faa..a8e6ed1e077ad 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -19,8 +19,8 @@ package org.apache.hudi.table; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java index 010e2a16af4a7..6ed38a952a63a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/clean/FlinkCleanActionExecutor.java @@ -21,8 +21,8 @@ import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java index 1d40b8e95a539..337e7cb269e3f 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/BaseFlinkCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -267,7 +267,7 @@ public Iterator> handleUpdate(String partitionPath, String fil return handleUpdateInternal(upsertHandle, fileId); } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException( @@ -277,11 +277,12 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups } // TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + if (upsertHandle.getPartitionPath() == null) { LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); + + upsertHandle.writeStatuses()); } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); + + return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); } protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java index a31ab4ee15d70..4b46f7fe7d6fa 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java index 57a87c412fa2a..185482642ead3 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeleteHelper.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java index 1752960321e0b..e0c47db26e484 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java index 96fcd06a64ecc..8e535475a465b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkInsertPreppedCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java index 24659c5fe40cd..7842e259f6473 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java index 89540804598af..a6ecd93199ec2 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkUpsertPreppedCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java index df106ce8d585c..191071e017a80 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java @@ -59,10 +59,8 @@ public List> deduplicateRecords(List> records, return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { @SuppressWarnings("unchecked") T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect - // everything - // so pick it from one of the records. - return new HoodieRecord(rec1.getKey(), reducedData); + HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); + return new HoodieRecord(reducedKey, reducedData); }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 2bcd3b2a7189e..8cc9b0df84238 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -29,6 +29,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -152,7 +153,7 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) for (SmallFile smallFile : smallFiles) { long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize, totalUnassignedInserts); - if (recordsToAppend > 0 && totalUnassignedInserts > 0) { + if (recordsToAppend > 0) { // create a new bucket or re-use an existing bucket int bucket; if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) { @@ -180,7 +181,11 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket); for (int b = 0; b < insertBuckets; b++) { bucketNumbers.add(totalBuckets); - recordsPerBucket.add(totalUnassignedInserts / insertBuckets); + if (b < insertBuckets - 1) { + recordsPerBucket.add(insertRecordsPerBucket); + } else { + recordsPerBucket.add(totalUnassignedInserts - (insertBuckets - 1) * insertRecordsPerBucket); + } BucketInfo bucketInfo = new BucketInfo(); bucketInfo.bucketType = BucketType.INSERT; bucketInfo.partitionPath = partitionPath; @@ -210,7 +215,8 @@ private Map> getSmallFilesForPartitions(List par Map> partitionSmallFilesMap = new HashMap<>(); if (partitionPaths != null && partitionPaths.size() > 0) { context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions"); - partitionSmallFilesMap = context.mapToPair(partitionPaths, partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath)), 0); + partitionSmallFilesMap = context.mapToPair(partitionPaths, + partitionPath -> new ImmutablePair<>(partitionPath, getSmallFiles(partitionPath)), 0); } return partitionSmallFilesMap; } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java index 28b713b24b9ab..47039a3adf20a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkCopyOnWriteRollbackActionExecutor.java @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -64,8 +64,8 @@ protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { @Override protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { - List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW( + context, table.getMetaClient().getBasePath(), config); return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java index 8cf91a21e3382..8ed66bddae005 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/FlinkMarkerBasedRollbackStrategy.java @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java index 612635da871f8..562acfd2038b0 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; @@ -48,8 +49,6 @@ import java.util.Objects; import java.util.stream.Collectors; -import scala.Tuple2; - /** * Performs Rollback of Hoodie Tables. */ @@ -106,13 +105,13 @@ Map maybeDeleteAndCollectStats(HoodieEngineContext c case DELETE_DATA_FILES_ONLY: { final Map filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete); - return new Tuple2<>(rollbackRequest.getPartitionPath(), + return new ImmutablePair<>(rollbackRequest.getPartitionPath(), HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath()) .withDeletedFileResults(filesToDeletedStatus).build()); } case DELETE_DATA_AND_LOG_FILES: { final Map filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete); - return new Tuple2<>(rollbackRequest.getPartitionPath(), + return new ImmutablePair<>(rollbackRequest.getPartitionPath(), HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath()) .withDeletedFileResults(filesToDeletedStatus).build()); } @@ -129,7 +128,7 @@ Map maybeDeleteAndCollectStats(HoodieEngineContext c if (doDelete) { Map header = generateHeader(instantToRollback.getTimestamp()); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); + writer.appendBlock(new HoodieCommandBlock(header)); } } catch (IOException | InterruptedException io) { throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io); @@ -150,7 +149,7 @@ Map maybeDeleteAndCollectStats(HoodieEngineContext c metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()), 1L ); - return new Tuple2<>(rollbackRequest.getPartitionPath(), + return new ImmutablePair<>(rollbackRequest.getPartitionPath(), HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath()) .withRollbackBlockAppendResults(filesToNumBlocksRollback).build()); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java index a96a14aa7800a..af57a9e5c36dc 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/FlinkUpgradeDowngrade.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java index 2d2e06e5979cb..3771ca1880182 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 1fa3ad0a32b7c..5db7de24c005b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -93,8 +93,7 @@ private static void recreateMarkerFiles(final String commitInstantTime, // generate rollback stats List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { - rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - table.getConfig().shouldAssumeDatePartitioning()); + rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } diff --git a/hudi-spark/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-flink-client/src/main/resources/log4j-surefire.properties similarity index 100% rename from hudi-spark/src/test/resources/log4j-surefire.properties rename to hudi-client/hudi-flink-client/src/main/resources/log4j-surefire.properties diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java index 41a8b96199395..05ec76208d235 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/common/TestHoodieFlinkEngineContext.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -30,8 +31,6 @@ import java.util.List; import java.util.Map; -import scala.Tuple2; - /** * Unit test against HoodieFlinkEngineContext. */ @@ -85,7 +84,7 @@ public void testMapToPair() { Map resultMap = context.mapToPair(mapList, x -> { String[] splits = x.split("_"); - return Tuple2.apply(splits[0], splits[1]); + return new ImmutablePair<>(splits[0], splits[1]); }, 2); Assertions.assertEquals(resultMap.get("spark"), resultMap.get("flink")); diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java new file mode 100644 index 0000000000000..0dc6997794764 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -0,0 +1,469 @@ +/* + * 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.index.bloom; + +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.HoodieKeyLookupHandle; +import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieFlinkClientTestHarness; +import org.apache.hudi.testutils.HoodieFlinkWriteableTestTable; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; + +import scala.Tuple2; + +import static java.util.Arrays.asList; +import static java.util.UUID.randomUUID; +import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Unit test against FlinkHoodieBloomIndex. + */ +//TODO merge code with Spark Bloom index tests. +public class TestFlinkHoodieBloomIndex extends HoodieFlinkClientTestHarness { + + private static final Schema SCHEMA = getSchemaFromResource(TestFlinkHoodieBloomIndex.class, "/exampleSchema.avsc", true); + private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}"; + + public static Stream configParams() { + Object[][] data = + new Object[][] {{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}}; + return Stream.of(data).map(Arguments::of); + } + + @BeforeEach + public void setUp() throws Exception { + initPath(); + initFileSystem(); + // We have some records to be tagged (two different partitions) + initMetaClient(); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { + return HoodieWriteConfig.newBuilder().withPath(basePath) + .withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning) + .bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking) + .bloomIndexKeysPerBucket(2).build()) + .build(); + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + FlinkHoodieBloomIndex index = new FlinkHoodieBloomIndex(config); + HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); + + // Create some partitions, and put some files + // "2016/01/21": 0 file + // "2016/04/01": 1 file (2_0_20160401010101.parquet) + // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet) + testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12"); + + RawTripTestPayload rowChange1 = + new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record1 = + new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + RawTripTestPayload rowChange2 = + new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record2 = + new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + RawTripTestPayload rowChange3 = + new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record3 = + new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + RawTripTestPayload rowChange4 = + new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"); + HoodieRecord record4 = + new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + + List partitions = asList("2016/01/21", "2016/04/01", "2015/03/12"); + List> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + // Still 0, as no valid commit + assertEquals(0, filesList.size()); + + testTable.addCommit("20160401010101").withInserts("2016/04/01", "2"); + testTable.addCommit("20150312101010").withInserts("2015/03/12", "1") + .withInserts("2015/03/12", "3", record1) + .withInserts("2015/03/12", "4", record2, record3, record4); + metaClient.reloadActiveTimeline(); + + filesList = index.loadInvolvedFiles(partitions, context, hoodieTable); + assertEquals(4, filesList.size()); + + if (rangePruning) { + // these files will not have the key ranges + assertNull(filesList.get(0)._2().getMaxRecordKey()); + assertNull(filesList.get(0)._2().getMinRecordKey()); + assertFalse(filesList.get(1)._2().hasKeyRanges()); + assertNotNull(filesList.get(2)._2().getMaxRecordKey()); + assertNotNull(filesList.get(2)._2().getMinRecordKey()); + assertTrue(filesList.get(3)._2().hasKeyRanges()); + + // no longer sorted, but should have same files. + + List> expected = + asList(new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")), + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")), + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")), + new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003"))); + assertEquals(expected, filesList); + } + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + FlinkHoodieBloomIndex index = new FlinkHoodieBloomIndex(config); + + final Map> partitionToFileIndexInfo = new HashMap<>(); + partitionToFileIndexInfo.put("2017/10/22", + asList(new BloomIndexFileInfo("f1"), new BloomIndexFileInfo("f2", "000", "000"), + new BloomIndexFileInfo("f3", "001", "003"), new BloomIndexFileInfo("f4", "002", "007"), + new BloomIndexFileInfo("f5", "009", "010"))); + + Map> partitionRecordKeyMap = new HashMap<>(); + asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), + new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004")) + .forEach(t -> { + List recordKeyList = partitionRecordKeyMap.getOrDefault(t._1, new ArrayList<>()); + recordKeyList.add(t._2); + partitionRecordKeyMap.put(t._1, recordKeyList); + }); + + List> comparisonKeyList = + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyMap); + + assertEquals(10, comparisonKeyList.size()); + java.util.Map> recordKeyToFileComps = comparisonKeyList.stream() + .collect(java.util.stream.Collectors.groupingBy(t -> t._2.getRecordKey(), java.util.stream.Collectors.mapping(t -> t._1, java.util.stream.Collectors.toList()))); + + assertEquals(4, recordKeyToFileComps.size()); + assertEquals(new java.util.HashSet<>(asList("f1", "f3", "f4")), new java.util.HashSet<>(recordKeyToFileComps.get("002"))); + assertEquals(new java.util.HashSet<>(asList("f1", "f3", "f4")), new java.util.HashSet<>(recordKeyToFileComps.get("003"))); + assertEquals(new java.util.HashSet<>(asList("f1", "f4")), new java.util.HashSet<>(recordKeyToFileComps.get("004"))); + assertEquals(new java.util.HashSet<>(asList("f1", "f4")), new java.util.HashSet<>(recordKeyToFileComps.get("005"))); + } + + @Test + public void testCheckUUIDsAgainstOneFile() throws Exception { + final String partition = "2016/01/31"; + // Create some records to use + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}"; + RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); + HoodieRecord record1 = + new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); + HoodieRecord record2 = + new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); + HoodieRecord record3 = + new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); + HoodieRecord record4 = + new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + + // We write record1, record2 to a parquet file, but the bloom filter contains (record1, + // record2, record3). + BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); + filter.add(record3.getRecordKey()); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA, filter); + String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2); + String filename = testTable.getBaseFileNameById(fileId); + + // The bloom filter contains 3 records + assertTrue(filter.mightContain(record1.getRecordKey())); + assertTrue(filter.mightContain(record2.getRecordKey())); + assertTrue(filter.mightContain(record3.getRecordKey())); + assertFalse(filter.mightContain(record4.getRecordKey())); + + // Compare with file + List uuids = asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey()); + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); + HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient); + HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId)); + List results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids, + new Path(java.nio.file.Paths.get(basePath, partition, filename).toString())); + assertEquals(results.size(), 2); + assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0") + || results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")); + assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0") + || results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")); + // TODO(vc): Need more coverage on actual filenames + // assertTrue(results.get(0)._2().equals(filename)); + // assertTrue(results.get(1)._2().equals(filename)); + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testTagLocationWithEmptyList(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) { + // We have some records to be tagged (two different partitions) + List records = new ArrayList<>(); + // Also create the metadata and config + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieFlinkTable table = HoodieFlinkTable.create(config, context, metaClient); + + // Let's tag + FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + + assertDoesNotThrow(() -> { + bloomIndex.tagLocation(records, context, table); + }, "EmptyList should not result in IllegalArgumentException: Positive number of slices required"); + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { + // We have some records to be tagged (two different partitions) + String rowKey1 = randomUUID().toString(); + String rowKey2 = randomUUID().toString(); + String rowKey3 = randomUUID().toString(); + String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + // place same row key under a different partition. + String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); + HoodieRecord record1 = + new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); + HoodieRecord record2 = + new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); + HoodieRecord record3 = + new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3); + RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); + HoodieRecord record4 = + new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4); + List records = asList(record1, record2, record3, record4); + + // Also create the metadata and config + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + HoodieFlinkTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); + + // Let's tag + FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + List taggedRecords = bloomIndex.tagLocation(records, context, hoodieTable); + + // Should not find any files + for (HoodieRecord record : taggedRecords) { + assertFalse(record.isCurrentLocationKnown()); + } + + // We create three parquet file, each having one record. (two different partitions) + String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1); + String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2); + String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4); + + metaClient.reloadActiveTimeline(); + + // We do the tag again + taggedRecords = bloomIndex.tagLocation(records, context, HoodieFlinkTable.create(config, context, metaClient)); + + // Check results + for (HoodieRecord record : taggedRecords) { + if (record.getRecordKey().equals(rowKey1)) { + if (record.getPartitionPath().equals("2015/01/31")) { + assertEquals(record.getCurrentLocation().getFileId(), fileId3); + } else { + assertEquals(record.getCurrentLocation().getFileId(), fileId1); + } + } else if (record.getRecordKey().equals(rowKey2)) { + assertEquals(record.getCurrentLocation().getFileId(), fileId2); + } else if (record.getRecordKey().equals(rowKey3)) { + assertFalse(record.isCurrentLocationKnown()); + } + } + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { + // We have some records to be tagged (two different partitions) + + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}"; + // record key same as recordStr2 + String recordStr4 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}"; + RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); + HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()); + HoodieRecord record1 = new HoodieRecord(key1, rowChange1); + RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); + HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()); + HoodieRecord record2 = new HoodieRecord(key2, rowChange2); + RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3); + HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()); + RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4); + HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()); + HoodieRecord record4 = new HoodieRecord(key4, rowChange4); + List keys = asList(key1, key2, key3, key4); + + // Also create the metadata and config + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + HoodieTable hoodieTable = HoodieFlinkTable.create(config, context, metaClient); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(hoodieTable, SCHEMA); + + // Let's tag + FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + List toTagRecords = new ArrayList<>(); + toTagRecords.add(new HoodieRecord(record4.getKey(), null)); + List taggedRecords = bloomIndex.tagLocation(toTagRecords, context, hoodieTable); + Map>> recordLocations = new HashMap<>(); + for (HoodieRecord taggedRecord : taggedRecords) { + recordLocations.put(taggedRecord.getKey(), taggedRecord.isCurrentLocationKnown() + ? Option.of(Pair.of(taggedRecord.getPartitionPath(), taggedRecord.getCurrentLocation().getFileId())) + : Option.empty()); + } + // Should not find any files + for (Option> record : recordLocations.values()) { + assertTrue(!record.isPresent()); + } + + // We create three parquet file, each having one record. (two different partitions) + String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1); + String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2); + String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4); + + // We do the tag again + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieFlinkTable.create(config, context, metaClient); + List toTagRecords1 = new ArrayList<>(); + for (HoodieKey key : keys) { + taggedRecords.add(new HoodieRecord(key, null)); + } + + taggedRecords = bloomIndex.tagLocation(toTagRecords1, context, hoodieTable); + recordLocations.clear(); + for (HoodieRecord taggedRecord : taggedRecords) { + recordLocations.put(taggedRecord.getKey(), taggedRecord.isCurrentLocationKnown() + ? Option.of(Pair.of(taggedRecord.getPartitionPath(), taggedRecord.getCurrentLocation().getFileId())) + : Option.empty()); + } + + // Check results + for (Map.Entry>> record : recordLocations.entrySet()) { + if (record.getKey().getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) { + assertTrue(record.getValue().isPresent()); + assertEquals(fileId1, record.getValue().get().getRight()); + } else if (record.getKey().getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) { + assertTrue(record.getValue().isPresent()); + if (record.getKey().getPartitionPath().equals("2015/01/31")) { + assertEquals(fileId3, record.getValue().get().getRight()); + } else { + assertEquals(fileId2, record.getValue().get().getRight()); + } + } else if (record.getKey().getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) { + assertFalse(record.getValue().isPresent()); + } + } + } + + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testBloomFilterFalseError(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception { + // We have two hoodie records + String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}"; + String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\"," + + "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}"; + + // We write record1 to a parquet file, using a bloom filter having both records + RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1); + HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1); + RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2); + HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2); + + BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); + filter.add(record2.getRecordKey()); + HoodieFlinkWriteableTestTable testTable = HoodieFlinkWriteableTestTable.of(metaClient, SCHEMA, filter); + String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1); + assertTrue(filter.mightContain(record1.getRecordKey())); + assertTrue(filter.mightContain(record2.getRecordKey())); + + // We do the tag + List records = asList(record1, record2); + HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable table = HoodieFlinkTable.create(config, context, metaClient); + + FlinkHoodieBloomIndex bloomIndex = new FlinkHoodieBloomIndex(config); + List taggedRecords = bloomIndex.tagLocation(records, context, table); + + // Check results + for (HoodieRecord record : taggedRecords) { + if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) { + assertEquals(record.getCurrentLocation().getFileId(), fileId); + } else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) { + assertFalse(record.isCurrentLocationKnown()); + } + } + } +} diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java new file mode 100644 index 0000000000000..171bab9fb2a31 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkClientTestHarness.java @@ -0,0 +1,230 @@ +/* + * 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.testutils; + +import org.apache.hudi.client.FlinkTaskContextSupplier; +import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.index.bloom.TestFlinkHoodieBloomIndex; + +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; + +/** + * The test harness for resource initialization and cleanup. + */ +public class HoodieFlinkClientTestHarness extends HoodieCommonTestHarness implements Serializable { + + protected static final Logger LOG = LogManager.getLogger(HoodieFlinkClientTestHarness.class); + private String testMethodName; + protected transient Configuration hadoopConf = null; + protected transient FileSystem fs; + protected transient MiniClusterWithClientResource flinkCluster = null; + protected transient HoodieFlinkEngineContext context = null; + protected transient ExecutorService executorService; + protected transient HoodieFlinkWriteClient writeClient; + protected transient HoodieTableFileSystemView tableView; + + protected final FlinkTaskContextSupplier supplier = new FlinkTaskContextSupplier(null); + + // dfs + protected transient HdfsTestService hdfsTestService; + protected transient MiniDFSCluster dfsCluster; + protected transient DistributedFileSystem dfs; + + @BeforeEach + public void setTestMethodName(TestInfo testInfo) { + if (testInfo.getTestMethod().isPresent()) { + testMethodName = testInfo.getTestMethod().get().getName(); + } else { + testMethodName = "Unknown"; + } + } + + protected void initFlinkMiniCluster() { + flinkCluster = new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberSlotsPerTaskManager(2) + .setNumberTaskManagers(1) + .build()); + } + + protected void initFileSystem() { + hadoopConf = new Configuration(); + initFileSystemWithConfiguration(hadoopConf); + context = new HoodieFlinkEngineContext(supplier); + } + + private void initFileSystemWithConfiguration(Configuration configuration) { + if (basePath == null) { + throw new IllegalStateException("The base path has not been initialized."); + } + fs = FSUtils.getFs(basePath, configuration); + if (fs instanceof LocalFileSystem) { + LocalFileSystem lfs = (LocalFileSystem) fs; + // With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream + // This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open + // So, for the tests, we enforce checksum verification to circumvent the problem + lfs.setVerifyChecksum(true); + } + } + + /** + * Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by + * {@code getTableType()}. + * + * @throws IOException + */ + protected void initMetaClient() throws IOException { + initMetaClient(getTableType()); + } + + protected void initMetaClient(HoodieTableType tableType) throws IOException { + if (basePath == null) { + throw new IllegalStateException("The base path has not been initialized."); + } + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); + } + + + /** + * Cleanups file system. + * + * @throws IOException + */ + protected void cleanupFileSystem() throws IOException { + if (fs != null) { + LOG.warn("Closing file-system instance used in previous test-run"); + fs.close(); + fs = null; + } + } + + /** + * Cleanups resource group for the subclasses of {@link TestFlinkHoodieBloomIndex}. + */ + public void cleanupResources() throws java.io.IOException { + cleanupClients(); + cleanupFlinkContexts(); + cleanupTestDataGenerator(); + cleanupFileSystem(); + cleanupDFS(); + cleanupExecutorService(); + System.gc(); + } + + protected void cleanupFlinkMiniCluster() { + if (flinkCluster != null) { + flinkCluster.after(); + flinkCluster = null; + } + } + + public static class SimpleTestSinkFunction implements SinkFunction { + + // must be static + public static List valuesList = new ArrayList<>(); + + @Override + public synchronized void invoke(HoodieRecord value, Context context) throws Exception { + valuesList.add(value); + } + } + + /** + * Cleanups hoodie clients. + */ + protected void cleanupClients() throws java.io.IOException { + if (metaClient != null) { + metaClient = null; + } + if (writeClient != null) { + writeClient.close(); + writeClient = null; + } + if (tableView != null) { + tableView.close(); + tableView = null; + } + } + + /** + * Cleanups the distributed file system. + * + * @throws IOException + */ + protected void cleanupDFS() throws java.io.IOException { + if (hdfsTestService != null) { + hdfsTestService.stop(); + dfsCluster.shutdown(); + hdfsTestService = null; + dfsCluster = null; + dfs = null; + } + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM + FileSystem.closeAll(); + } + + /** + * Cleanups the executor service. + */ + protected void cleanupExecutorService() { + if (this.executorService != null) { + this.executorService.shutdownNow(); + this.executorService = null; + } + } + + /** + * Cleanups Flink contexts. + */ + protected void cleanupFlinkContexts() { + if (context != null) { + LOG.info("Closing flink engine context used in previous test-case"); + context = null; + } + } +} diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java new file mode 100644 index 0000000000000..60ae294e6a927 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java @@ -0,0 +1,136 @@ +/* + * 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.testutils; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HoodieFlinkWriteableTestTable extends HoodieWriteableTestTable { + private static final Logger LOG = LogManager.getLogger(HoodieFlinkWriteableTestTable.class); + + private HoodieFlinkWriteableTestTable(String basePath, org.apache.hadoop.fs.FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { + super(basePath, fs, metaClient, schema, filter); + } + + public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { + return new HoodieFlinkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter); + } + + public static HoodieFlinkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { + BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); + return of(metaClient, schema, filter); + } + + public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + return of(metaClient, schema); + } + + public static HoodieFlinkWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) { + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + return of(metaClient, schema, filter); + } + + @Override + public HoodieFlinkWriteableTestTable addCommit(String instantTime) throws Exception { + return (HoodieFlinkWriteableTestTable) super.addCommit(instantTime); + } + + @Override + public HoodieFlinkWriteableTestTable forCommit(String instantTime) { + return (HoodieFlinkWriteableTestTable) super.forCommit(instantTime); + } + + public String getFileIdWithInserts(String partition) throws Exception { + return getFileIdWithInserts(partition, new HoodieRecord[0]); + } + + public String getFileIdWithInserts(String partition, HoodieRecord... records) throws Exception { + return getFileIdWithInserts(partition, Arrays.asList(records)); + } + + public String getFileIdWithInserts(String partition, List records) throws Exception { + String fileId = java.util.UUID.randomUUID().toString(); + withInserts(partition, fileId, records); + return fileId; + } + + public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId) throws Exception { + return withInserts(partition, fileId, new HoodieRecord[0]); + } + + public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception { + return withInserts(partition, fileId, Arrays.asList(records)); + } + + public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId, List records) throws Exception { + return (HoodieFlinkWriteableTestTable) withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null)); + } + + public HoodieFlinkWriteableTestTable withLogAppends(List records) throws Exception { + for (List groupedRecords: records.stream().collect(Collectors.groupingBy(HoodieRecord::getCurrentLocation)).values()) { + appendRecordsToLogFile(groupedRecords); + } + return this; + } + + private void appendRecordsToLogFile(List groupedRecords) throws Exception { + String partitionPath = groupedRecords.get(0).getPartitionPath(); + HoodieRecordLocation location = groupedRecords.get(0).getCurrentLocation(); + try (HoodieLogFormat.Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(basePath, partitionPath)) + .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId()) + .overBaseCommit(location.getInstantTime()).withFs(fs).build()) { + Map header = new java.util.HashMap<>(); + header.put(HeaderMetadataType.INSTANT_TIME, location.getInstantTime()); + header.put(HeaderMetadataType.SCHEMA, schema.toString()); + logWriter.appendBlock(new HoodieAvroDataBlock(groupedRecords.stream().map(r -> { + try { + GenericRecord val = (GenericRecord) r.getData().getInsertValue(schema).get(); + HoodieAvroUtils.addHoodieKeyToRecord(val, r.getRecordKey(), r.getPartitionPath(), ""); + return (org.apache.avro.generic.IndexedRecord) val; + } catch (java.io.IOException e) { + LOG.warn("Failed to convert record " + r.toString(), e); + return null; + } + }).collect(Collectors.toList()), header)); + } + } +} diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index 6429adedc6e12..0ef741f924772 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -15,17 +15,16 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi-client org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 hudi-java-client - ${parent.version} + 0.8.0-SNAPSHOT hudi-java-client jar @@ -38,6 +37,12 @@ ${parent.version} + + + org.apache.parquet + parquet-avro + + org.apache.hudi @@ -55,6 +60,12 @@ test-jar test + + org.apache.hudi + hudi-hadoop-mr + ${project.version} + test + diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java new file mode 100644 index 0000000000000..a162663d906d5 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +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.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.JavaHoodieIndex; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieJavaTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HoodieJavaWriteClient extends + AbstractHoodieWriteClient>, List, List> { + + public HoodieJavaWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { + super(context, clientConfig); + } + + public HoodieJavaWriteClient(HoodieEngineContext context, + HoodieWriteConfig writeConfig, + boolean rollbackPending, + Option timelineService) { + super(context, writeConfig, rollbackPending, timelineService); + } + + @Override + public List> filterExists(List> hoodieRecords) { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieJavaTable table = HoodieJavaTable.create(config, (HoodieJavaEngineContext) context); + Timer.Context indexTimer = metrics.getIndexCtx(); + List> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table); + metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop())); + return recordsWithLocation.stream().filter(v1 -> !v1.isCurrentLocationKnown()).collect(Collectors.toList()); + } + + @Override + protected HoodieIndex>, List, List> createIndex(HoodieWriteConfig writeConfig) { + return JavaHoodieIndex.createIndex(config); + } + + @Override + public boolean commit(String instantTime, + List writeStatuses, + Option> extraMetadata, + String commitActionType, + Map> partitionToReplacedFileIds) { + List writeStats = writeStatuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()); + return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); + } + + @Override + protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, + Configuration hadoopConf) { + return HoodieJavaTable.create(config, context); + } + + @Override + public List upsert(List> records, + String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); + table.validateUpsertSchema(); + preWrite(instantTime, WriteOperationType.UPSERT); + HoodieWriteMetadata> result = table.upsert(context, instantTime, records); + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); + } + return postWrite(result, instantTime, table); + } + + @Override + public List upsertPreppedRecords(List> preppedRecords, + String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); + table.validateUpsertSchema(); + preWrite(instantTime, WriteOperationType.UPSERT_PREPPED); + HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); + return postWrite(result, instantTime, table); + } + + @Override + public List insert(List> records, String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.INSERT, instantTime); + table.validateUpsertSchema(); + preWrite(instantTime, WriteOperationType.INSERT); + HoodieWriteMetadata> result = table.insert(context, instantTime, records); + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); + } + return postWrite(result, instantTime, table); + } + + @Override + public List insertPreppedRecords(List> preppedRecords, + String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); + table.validateInsertSchema(); + preWrite(instantTime, WriteOperationType.INSERT_PREPPED); + HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); + return postWrite(result, instantTime, table); + } + + @Override + public List bulkInsert(List> records, + String instantTime) { + throw new HoodieNotSupportedException("BulkInsert is not supported in HoodieJavaClient"); + } + + @Override + public List bulkInsert(List> records, + String instantTime, + Option>>> userDefinedBulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsert is not supported in HoodieJavaClient"); + } + + @Override + public List bulkInsertPreppedRecords(List> preppedRecords, + String instantTime, + Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsertPreppedRecords is not supported in HoodieJavaClient"); + } + + @Override + public List delete(List keys, + String instantTime) { + HoodieTable>, List, List> table = + getTableAndInitCtx(WriteOperationType.DELETE, instantTime); + preWrite(instantTime, WriteOperationType.DELETE); + HoodieWriteMetadata> result = table.delete(context,instantTime, keys); + return postWrite(result, instantTime, table); + } + + @Override + protected List postWrite(HoodieWriteMetadata> result, + String instantTime, + HoodieTable>, List, List> hoodieTable) { + if (result.getIndexLookupDuration().isPresent()) { + metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); + } + if (result.isCommitted()) { + // Perform post commit operations. + if (result.getFinalizeDuration().isPresent()) { + metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(), + result.getWriteStats().get().size()); + } + + postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty()); + + emitCommitMetrics(instantTime, result.getCommitMetadata().get(), hoodieTable.getMetaClient().getCommitActionType()); + } + return result.getWriteStatuses(); + } + + @Override + public void commitCompaction(String compactionInstantTime, + List writeStatuses, + Option> extraMetadata) throws IOException { + throw new HoodieNotSupportedException("CommitCompaction is not supported in HoodieJavaClient"); + } + + @Override + protected void completeCompaction(HoodieCommitMetadata metadata, + List writeStatuses, + HoodieTable>, List, List> table, + String compactionCommitTime) { + throw new HoodieNotSupportedException("CompleteCompaction is not supported in HoodieJavaClient"); + } + + @Override + protected List compact(String compactionInstantTime, + boolean shouldComplete) { + throw new HoodieNotSupportedException("Compact is not supported in HoodieJavaClient"); + } + + @Override + public HoodieWriteMetadata> cluster(final String clusteringInstant, final boolean shouldComplete) { + throw new HoodieNotSupportedException("Cluster is not supported in HoodieJavaClient"); + } + + @Override + protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { + HoodieTableMetaClient metaClient = createMetaClient(true); + // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); + return getTableAndInitCtx(metaClient, operationType); + } + + private HoodieTable>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) { + if (operationType == WriteOperationType.DELETE) { + setWriteSchemaForDeletes(metaClient); + } + // Create a Hoodie table which encapsulated the commits and files visible + HoodieJavaTable table = HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient); + if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) { + writeTimer = metrics.getCommitCtx(); + } else { + writeTimer = metrics.getDeltaCommitCtx(); + } + return table; + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java index a04a18b190964..013e094036b6a 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java @@ -19,13 +19,16 @@ package org.apache.hudi.client.common; import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.client.common.function.SerializableConsumer; -import org.apache.hudi.client.common.function.SerializableFunction; -import org.apache.hudi.client.common.function.SerializablePairFunction; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.function.SerializableConsumer; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; -import scala.Tuple2; +import org.apache.hudi.common.util.collection.Pair; import java.util.List; import java.util.Map; @@ -33,16 +36,20 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingFlatMapWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingForeachWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingMapToPairWrapper; -import static org.apache.hudi.client.common.function.FunctionWrapper.throwingMapWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; /** * A java engine implementation of HoodieEngineContext. */ public class HoodieJavaEngineContext extends HoodieEngineContext { + public HoodieJavaEngineContext(Configuration conf) { + this(conf, new JavaTaskContextSupplier()); + } + public HoodieJavaEngineContext(Configuration conf, TaskContextSupplier taskContextSupplier) { super(new SerializableConfiguration(conf), taskContextSupplier); } @@ -65,7 +72,7 @@ public void foreach(List data, SerializableConsumer consumer, int para @Override public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { return data.stream().map(throwingMapToPairWrapper(func)).collect( - Collectors.toMap(Tuple2::_1, Tuple2::_2, (oldVal, newVal) -> newVal) + Collectors.toMap(Pair::getLeft, Pair::getRight, (oldVal, newVal) -> newVal) ); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/JavaTaskContextSupplier.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/JavaTaskContextSupplier.java new file mode 100644 index 0000000000000..628201ccc25ae --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/JavaTaskContextSupplier.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.common; + +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.util.Option; + +import java.util.function.Supplier; + +public class JavaTaskContextSupplier extends TaskContextSupplier { + @Override + public Supplier getPartitionIdSupplier() { + return () -> 0; + } + + @Override + public Supplier getStageIdSupplier() { + return () -> 0; + } + + @Override + public Supplier getAttemptIdSupplier() { + return () -> 0L; + } + + @Override + public Option getProperty(EngineProperty prop) { + return Option.empty(); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java new file mode 100644 index 0000000000000..f91dd5019a275 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java @@ -0,0 +1,80 @@ +/* + * 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.execution; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.WriteHandleFactory; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; + +import java.util.Iterator; +import java.util.List; + +public class JavaLazyInsertIterable extends HoodieLazyInsertIterable { + public JavaLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier) { + super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); + } + + public JavaLazyInsertIterable(Iterator> recordItr, + boolean areRecordsSorted, + HoodieWriteConfig config, + String instantTime, + HoodieTable hoodieTable, + String idPrefix, + TaskContextSupplier taskContextSupplier, + WriteHandleFactory writeHandleFactory) { + super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory); + } + + @Override + protected List computeNext() { + // Executor service used for launching writer thread. + BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = + null; + try { + final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + bufferedIteratorExecutor = + new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(inputItr), Option.of(getInsertHandler()), getTransformFunction(schema)); + final List result = bufferedIteratorExecutor.execute(); + assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining(); + return result; + } catch (Exception e) { + throw new HoodieException(e); + } finally { + if (null != bufferedIteratorExecutor) { + bufferedIteratorExecutor.shutdownNow(); + } + } + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java new file mode 100644 index 0000000000000..0239ee903d1bf --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaHoodieIndex.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index; + +import org.apache.hudi.ApiMaturityLevel; +import org.apache.hudi.PublicAPIMethod; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +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.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.table.HoodieTable; + +import java.util.List; + +public abstract class JavaHoodieIndex extends HoodieIndex>, List, List> { + protected JavaHoodieIndex(HoodieWriteConfig config) { + super(config); + } + + public static JavaHoodieIndex createIndex(HoodieWriteConfig config) { + // first use index class config to create index. + if (!StringUtils.isNullOrEmpty(config.getIndexClass())) { + Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config); + if (!(instance instanceof HoodieIndex)) { + throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex"); + } + return (JavaHoodieIndex) instance; + } + + // TODO more indexes to be added + switch (config.getIndexType()) { + case INMEMORY: + return new JavaInMemoryHashIndex(config); + default: + throw new HoodieIndexException("Unsupported index type " + config.getIndexType()); + } + } + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public abstract List updateLocation(List writeStatuses, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; + + @Override + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public abstract List> tagLocation(List> records, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) throws HoodieIndexException; +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java new file mode 100644 index 0000000000000..8a7197759aa0a --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/index/JavaInMemoryHashIndex.java @@ -0,0 +1,120 @@ +/* + * 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.index; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +/** + * Hoodie Index implementation backed by an in-memory Hash map. + *

+ * ONLY USE FOR LOCAL TESTING + */ +@SuppressWarnings("checkstyle:LineLength") +public class JavaInMemoryHashIndex extends JavaHoodieIndex { + + private static ConcurrentMap recordLocationMap; + + public JavaInMemoryHashIndex(HoodieWriteConfig config) { + super(config); + synchronized (JavaInMemoryHashIndex.class) { + if (recordLocationMap == null) { + recordLocationMap = new ConcurrentHashMap<>(); + } + } + } + + @Override + public List> tagLocation(List> records, HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) { + List> taggedRecords = new ArrayList<>(); + records.stream().forEach(record -> { + if (recordLocationMap.containsKey(record.getKey())) { + record.unseal(); + record.setCurrentLocation(recordLocationMap.get(record.getKey())); + record.seal(); + } + taggedRecords.add(record); + }); + return taggedRecords; + } + + @Override + public List updateLocation(List writeStatusList, + HoodieEngineContext context, + HoodieTable>, List, List> hoodieTable) { + return writeStatusList.stream().map(writeStatus -> { + for (HoodieRecord record : writeStatus.getWrittenRecords()) { + if (!writeStatus.isErrored(record.getKey())) { + HoodieKey key = record.getKey(); + Option newLocation = record.getNewLocation(); + if (newLocation.isPresent()) { + recordLocationMap.put(key, newLocation.get()); + } else { + // Delete existing index for a deleted record + recordLocationMap.remove(key); + } + } + } + return writeStatus; + }).collect(Collectors.toList()); + } + + @Override + public boolean rollbackCommit(String instantTime) { + return true; + } + + /** + * Only looks up by recordKey. + */ + @Override + public boolean isGlobal() { + return true; + } + + /** + * Mapping is available in HBase already. + */ + @Override + public boolean canIndexLogFiles() { + return true; + } + + /** + * Index needs to be explicitly updated after storage write. + */ + @Override + public boolean isImplicitWithStorage() { + return false; + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java new file mode 100644 index 0000000000000..9895df3a3e254 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -0,0 +1,195 @@ +/* + * 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; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +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.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.clean.JavaCleanActionExecutor; +import org.apache.hudi.table.action.commit.JavaDeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaInsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaUpsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.JavaUpsertPreppedCommitActionExecutor; + +import java.util.List; +import java.util.Map; + +public class HoodieJavaCopyOnWriteTable extends HoodieJavaTable { + protected HoodieJavaCopyOnWriteTable(HoodieWriteConfig config, + HoodieEngineContext context, + HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + @Override + public HoodieWriteMetadata> upsert(HoodieEngineContext context, + String instantTime, + List> records) { + return new JavaUpsertCommitActionExecutor<>(context, config, + this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> insert(HoodieEngineContext context, + String instantTime, + List> records) { + return new JavaInsertCommitActionExecutor<>(context, config, + this, instantTime, records).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, + String instantTime, + List> records, + Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsert is not supported yet"); + } + + @Override + public HoodieWriteMetadata> delete(HoodieEngineContext context, + String instantTime, + List keys) { + return new JavaDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute(); + } + + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + throw new HoodieNotSupportedException("Delete partitions is not supported yet"); + } + + @Override + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, + String instantTime, + List> preppedRecords) { + return new JavaUpsertPreppedCommitActionExecutor<>((HoodieJavaEngineContext) context, config, + this, instantTime, preppedRecords).execute(); + + } + + @Override + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, + String instantTime, + List> preppedRecords) { + return new JavaInsertPreppedCommitActionExecutor<>((HoodieJavaEngineContext) context, config, + this, instantTime, preppedRecords).execute(); + } + + @Override + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, + String instantTime, + List> preppedRecords, + Option>>> bulkInsertPartitioner) { + throw new HoodieNotSupportedException("BulkInsertPrepped is not supported yet"); + } + + @Override + public HoodieWriteMetadata> insertOverwrite(HoodieEngineContext context, + String instantTime, + List> records) { + throw new HoodieNotSupportedException("InsertOverwrite is not supported yet"); + } + + @Override + public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, + String instantTime, + List> records) { + throw new HoodieNotSupportedException("InsertOverwrite is not supported yet"); + } + + @Override + public Option scheduleCompaction(HoodieEngineContext context, + String instantTime, + Option> extraMetadata) { + throw new HoodieNotSupportedException("ScheduleCompaction is not supported yet"); + } + + @Override + public HoodieWriteMetadata> compact(HoodieEngineContext context, + String compactionInstantTime) { + throw new HoodieNotSupportedException("Compact is not supported yet"); + } + + @Override + public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + + @Override + public HoodieWriteMetadata> cluster(final HoodieEngineContext context, final String clusteringInstantTime) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + + @Override + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, + Option> extraMetadata) { + throw new HoodieNotSupportedException("Bootstrap is not supported yet"); + } + + @Override + public void rollbackBootstrap(HoodieEngineContext context, + String instantTime) { + throw new HoodieNotSupportedException("RollbackBootstrap is not supported yet"); + } + + @Override + public HoodieCleanMetadata clean(HoodieEngineContext context, + String cleanInstantTime) { + return new JavaCleanActionExecutor(context, config, this, cleanInstantTime).execute(); + } + + @Override + public HoodieRollbackMetadata rollback(HoodieEngineContext context, + String rollbackInstantTime, + HoodieInstant commitInstant, + boolean deleteInstants) { + throw new HoodieNotSupportedException("Rollback is not supported yet"); + } + + @Override + public HoodieSavepointMetadata savepoint(HoodieEngineContext context, + String instantToSavepoint, + String user, + String comment) { + throw new HoodieNotSupportedException("Savepoint is not supported yet"); + } + + @Override + public HoodieRestoreMetadata restore(HoodieEngineContext context, + String restoreInstantTime, + String instantToRestore) { + throw new HoodieNotSupportedException("Restore is not supported yet"); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java new file mode 100644 index 0000000000000..4995af0d6acf2 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -0,0 +1,31 @@ +/* + * 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; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieWriteConfig; + +public class HoodieJavaMergeOnReadTable extends HoodieJavaCopyOnWriteTable { + protected HoodieJavaMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + // TODO not support yet. +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java new file mode 100644 index 0000000000000..bd8f9547c27b5 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -0,0 +1,72 @@ +/* + * 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; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +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.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.index.JavaHoodieIndex; +import org.apache.hudi.index.HoodieIndex; + +import java.util.List; + +public abstract class HoodieJavaTable + extends HoodieTable>, List, List> { + protected HoodieJavaTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) { + super(config, context, metaClient); + } + + public static HoodieJavaTable create(HoodieWriteConfig config, HoodieEngineContext context) { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient( + context.getHadoopConf().get(), + config.getBasePath(), + true, + config.getConsistencyGuardConfig(), + Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())) + ); + return HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient); + } + + public static HoodieJavaTable create(HoodieWriteConfig config, + HoodieJavaEngineContext context, + HoodieTableMetaClient metaClient) { + switch (metaClient.getTableType()) { + case COPY_ON_WRITE: + return new HoodieJavaCopyOnWriteTable<>(config, context, metaClient); + case MERGE_ON_READ: + throw new HoodieNotSupportedException("MERGE_ON_READ is not supported yet"); + default: + throw new HoodieException("Unsupported table type :" + metaClient.getTableType()); + } + } + + @Override + protected HoodieIndex>, List, List> getIndex(HoodieWriteConfig config, HoodieEngineContext context) { + return JavaHoodieIndex.createIndex(config); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java new file mode 100644 index 0000000000000..0ca73d40e84dd --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/clean/JavaCleanActionExecutor.java @@ -0,0 +1,130 @@ +/* + * 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.clean; + +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.CleanFileInfo; +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.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class JavaCleanActionExecutor extends + BaseCleanActionExecutor>, List, List> { + + private static final Logger LOG = LogManager.getLogger(JavaCleanActionExecutor.class); + + public JavaCleanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + String instantTime) { + super(context, config, table, instantTime); + } + + @Override + List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) { + + Iterator> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() + .flatMap(x -> x.getValue().stream().map(y -> new ImmutablePair<>(x.getKey(), new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile())))).iterator(); + + Stream> partitionCleanStats = + deleteFilesFunc(filesToBeDeletedPerPartition, table) + .collect(Collectors.groupingBy(Pair::getLeft)) + .entrySet().stream() + .map(x -> new ImmutablePair(x.getKey(), x.getValue().stream().map(y -> y.getRight()).reduce(PartitionCleanStat::merge).get())); + + Map partitionCleanStatsMap = partitionCleanStats + .collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + + // Return PartitionCleanStat for each partition passed. + return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> { + PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath) + ? partitionCleanStatsMap.get(partitionPath) + : new PartitionCleanStat(partitionPath); + HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain(); + return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath) + .withEarliestCommitRetained(Option.ofNullable( + actionInstant != null + ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()), + actionInstant.getAction(), actionInstant.getTimestamp()) + : null)) + .withDeletePathPattern(partitionCleanStat.deletePathPatterns()) + .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles()) + .withFailedDeletes(partitionCleanStat.failedDeleteFiles()) + .withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns()) + .withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles()) + .withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles()) + .build(); + }).collect(Collectors.toList()); + } + + private static Stream> deleteFilesFunc(Iterator> iter, HoodieTable table) { + Map partitionCleanStatMap = new HashMap<>(); + FileSystem fs = table.getMetaClient().getFs(); + + while (iter.hasNext()) { + Pair partitionDelFileTuple = iter.next(); + String partitionPath = partitionDelFileTuple.getLeft(); + Path deletePath = new Path(partitionDelFileTuple.getRight().getFilePath()); + String deletePathStr = deletePath.toString(); + Boolean deletedFileResult = null; + try { + deletedFileResult = deleteFileAndGetResult(fs, deletePathStr); + } catch (IOException e) { + LOG.error("Delete file failed"); + } + if (!partitionCleanStatMap.containsKey(partitionPath)) { + partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath)); + } + boolean isBootstrapBasePathFile = partitionDelFileTuple.getRight().isBootstrapBaseFile(); + PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath); + if (isBootstrapBasePathFile) { + // For Bootstrap Base file deletions, store the full file path. + partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true); + partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true); + } else { + partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false); + partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false); + } + } + return partitionCleanStatMap.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue())); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java new file mode 100644 index 0000000000000..3e0b80c5d5a41 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -0,0 +1,328 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.execution.JavaLazyInsertIterable; +import org.apache.hudi.io.CreateHandleFactory; +import org.apache.hudi.io.HoodieMergeHandle; +import org.apache.hudi.io.HoodieSortedMergeHandle; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public abstract class BaseJavaCommitActionExecutor extends + BaseCommitActionExecutor>, List, List, HoodieWriteMetadata> { + + private static final Logger LOG = LogManager.getLogger(BaseJavaCommitActionExecutor.class); + + public BaseJavaCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + WriteOperationType operationType) { + super(context, config, table, instantTime, operationType, Option.empty()); + } + + public BaseJavaCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + WriteOperationType operationType, + Option extraMetadata) { + super(context, config, table, instantTime, operationType, extraMetadata); + } + + @Override + public HoodieWriteMetadata> execute(List> inputRecords) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + + WorkloadProfile profile = null; + if (isWorkloadProfileNeeded()) { + profile = new WorkloadProfile(buildProfile(inputRecords)); + LOG.info("Workload profile :" + profile); + try { + saveWorkloadProfileMetadataToInflight(profile, instantTime); + } catch (Exception e) { + HoodieTableMetaClient metaClient = table.getMetaClient(); + HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime); + try { + if (!metaClient.getFs().exists(new Path(metaClient.getMetaPath(), inflightInstant.getFileName()))) { + throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e); + } + } catch (IOException ex) { + LOG.error("Check file exists failed"); + throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", ex); + } + } + } + + final Partitioner partitioner = getPartitioner(profile); + Map>> partitionedRecords = partition(inputRecords, partitioner); + + List writeStatuses = new LinkedList<>(); + partitionedRecords.forEach((partition, records) -> { + if (WriteOperationType.isChangingRecords(operationType)) { + handleUpsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll); + } else { + handleInsertPartition(instantTime, partition, records.iterator(), partitioner).forEachRemaining(writeStatuses::addAll); + } + }); + updateIndex(writeStatuses, result); + return result; + } + + protected void updateIndex(List writeStatuses, HoodieWriteMetadata> result) { + Instant indexStartTime = Instant.now(); + // Update the index back + List statuses = table.getIndex().updateLocation(writeStatuses, context, table); + result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); + result.setWriteStatuses(statuses); + } + + @Override + protected String getCommitActionType() { + return table.getMetaClient().getCommitActionType(); + } + + private Partitioner getPartitioner(WorkloadProfile profile) { + if (WriteOperationType.isChangingRecords(operationType)) { + return getUpsertPartitioner(profile); + } else { + return getInsertPartitioner(profile); + } + } + + private Map>> partition(List> dedupedRecords, Partitioner partitioner) { + Map>, HoodieRecord>>> partitionedMidRecords = dedupedRecords + .stream() + .map(record -> Pair.of(Pair.of(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)) + .collect(Collectors.groupingBy(x -> partitioner.getPartition(x.getLeft()))); + Map>> results = new LinkedHashMap<>(); + partitionedMidRecords.forEach((key, value) -> results.put(key, value.stream().map(x -> x.getRight()).collect(Collectors.toList()))); + return results; + } + + protected Pair, WorkloadStat> buildProfile(List> inputRecords) { + HashMap partitionPathStatMap = new HashMap<>(); + WorkloadStat globalStat = new WorkloadStat(); + + Map>, Long> partitionLocationCounts = inputRecords + .stream() + .map(record -> Pair.of( + Pair.of(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) + .collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting())); + + for (Map.Entry>, Long> e : partitionLocationCounts.entrySet()) { + String partitionPath = e.getKey().getLeft(); + Long count = e.getValue(); + Option locOption = e.getKey().getRight(); + + if (!partitionPathStatMap.containsKey(partitionPath)) { + partitionPathStatMap.put(partitionPath, new WorkloadStat()); + } + + if (locOption.isPresent()) { + // update + partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count); + globalStat.addUpdates(locOption.get(), count); + } else { + // insert + partitionPathStatMap.get(partitionPath).addInserts(count); + globalStat.addInserts(count); + } + } + return Pair.of(partitionPathStatMap, globalStat); + } + + @Override + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + commit(extraMetadata, result, result.getWriteStatuses().stream().map(WriteStatus::getStat).collect(Collectors.toList())); + } + + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { + String actionType = getCommitActionType(); + LOG.info("Committing " + instantTime + ", action Type " + actionType); + result.setCommitted(true); + result.setWriteStats(writeStats); + // Finalize write + finalizeWrite(instantTime, writeStats, result); + + try { + LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + LOG.info("Committed " + instantTime); + result.setCommitMetadata(Option.of(metadata)); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, + e); + } + } + + protected Map> getPartitionToReplacedFileIds(List writeStatuses) { + return Collections.emptyMap(); + } + + @Override + protected boolean isWorkloadProfileNeeded() { + return true; + } + + @SuppressWarnings("unchecked") + protected Iterator> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr, + Partitioner partitioner) { + UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner; + BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); + BucketType btype = binfo.bucketType; + try { + if (btype.equals(BucketType.INSERT)) { + return handleInsert(binfo.fileIdPrefix, recordItr); + } else if (btype.equals(BucketType.UPDATE)) { + return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr); + } else { + throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition); + } + } catch (Throwable t) { + String msg = "Error upserting bucketType " + btype + " for partition :" + partition; + LOG.error(msg, t); + throw new HoodieUpsertException(msg, t); + } + } + + protected Iterator> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, + Partitioner partitioner) { + return handleUpsertPartition(instantTime, partition, recordItr, partitioner); + } + + @Override + public Iterator> handleUpdate(String partitionPath, String fileId, + Iterator> recordItr) + throws IOException { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition with fileId => " + fileId); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + // these are updates + HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); + return handleUpdateInternal(upsertHandle, fileId); + } + + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) + throws IOException { + if (upsertHandle.getOldFilePath() == null) { + throw new HoodieUpsertException( + "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); + } else { + JavaMergeHelper.newInstance().runMerge(table, upsertHandle); + } + + List statuses = upsertHandle.writeStatuses(); + if (upsertHandle.getPartitionPath() == null) { + LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " + statuses); + } + return Collections.singletonList(statuses).iterator(); + } + + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { + if (table.requireSortedRecords()) { + return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + } else { + return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier); + } + } + + protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, + Map> keyToNewRecords, + HoodieBaseFile dataFileToBeMerged) { + return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords, + partitionPath, fileId, dataFileToBeMerged, taskContextSupplier); + } + + @Override + public Iterator> handleInsert(String idPfx, Iterator> recordItr) + throws Exception { + // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records + if (!recordItr.hasNext()) { + LOG.info("Empty partition"); + return Collections.singletonList((List) Collections.EMPTY_LIST).iterator(); + } + return new JavaLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, + taskContextSupplier, new CreateHandleFactory<>()); + } + + /** + * Provides a partitioner to perform the upsert operation, based on the workload profile. + */ + public Partitioner getUpsertPartitioner(WorkloadProfile profile) { + if (profile == null) { + throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); + } + return new UpsertPartitioner(profile, context, table, config); + } + + /** + * Provides a partitioner to perform the insert operation, based on the workload profile. + */ + public Partitioner getInsertPartitioner(WorkloadProfile profile) { + return getUpsertPartitioner(profile); + } + +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java new file mode 100644 index 0000000000000..72c2332645cf3 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteCommitActionExecutor.java @@ -0,0 +1,46 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class JavaDeleteCommitActionExecutor> extends BaseJavaCommitActionExecutor { + private final List keys; + + public JavaDeleteCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List keys) { + super(context, config, table, instantTime, WriteOperationType.DELETE); + this.keys = keys; + } + + @Override + public HoodieWriteMetadata> execute() { + return JavaDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java new file mode 100644 index 0000000000000..a907c9fd9760e --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaDeleteHelper.java @@ -0,0 +1,125 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.EmptyHoodieRecordPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.time.Duration; +import java.time.Instant; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.stream.Collectors; + +@SuppressWarnings("checkstyle:LineLength") +public class JavaDeleteHelper extends + AbstractDeleteHelper>, List, List, R> { + + private JavaDeleteHelper() { + } + + private static class DeleteHelperHolder { + private static final JavaDeleteHelper JAVA_DELETE_HELPER = new JavaDeleteHelper(); + } + + public static JavaDeleteHelper newInstance() { + return DeleteHelperHolder.JAVA_DELETE_HELPER; + } + + @Override + public List deduplicateKeys(List keys, + HoodieTable>, List, List> table, + int parallelism) { + boolean isIndexingGlobal = table.getIndex().isGlobal(); + if (isIndexingGlobal) { + HashSet recordKeys = keys.stream().map(HoodieKey::getRecordKey).collect(Collectors.toCollection(HashSet::new)); + List deduplicatedKeys = new LinkedList<>(); + keys.forEach(x -> { + if (recordKeys.contains(x.getRecordKey())) { + deduplicatedKeys.add(x); + } + }); + return deduplicatedKeys; + } else { + HashSet set = new HashSet<>(keys); + keys.clear(); + keys.addAll(set); + return keys; + } + } + + @Override + public HoodieWriteMetadata> execute(String instantTime, + List keys, + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, List, List> table, + BaseCommitActionExecutor>, List, List, R> deleteExecutor) { + try { + HoodieWriteMetadata> result = null; + List dedupedKeys = keys; + final int parallelism = config.getDeleteShuffleParallelism(); + if (config.shouldCombineBeforeDelete()) { + // De-dupe/merge if needed + dedupedKeys = deduplicateKeys(keys, table, parallelism); + } + + List> dedupedRecords = + dedupedKeys.stream().map(key -> new HoodieRecord<>(key, new EmptyHoodieRecordPayload())).collect(Collectors.toList()); + Instant beginTag = Instant.now(); + // perform index look up to get existing location of records + List> taggedRecords = + table.getIndex().tagLocation(dedupedRecords, context, table); + Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); + + // filter out non existent keys/records + List> taggedValidRecords = taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).collect(Collectors.toList()); + if (!taggedValidRecords.isEmpty()) { + result = deleteExecutor.execute(taggedValidRecords); + result.setIndexLookupDuration(tagLocationDuration); + } else { + // if entire set of keys are non existent + deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + result = new HoodieWriteMetadata<>(); + result.setWriteStatuses(Collections.EMPTY_LIST); + deleteExecutor.commitOnAutoCommit(result); + } + return result; + } catch (Throwable e) { + if (e instanceof HoodieUpsertException) { + throw (HoodieUpsertException) e; + } + throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e); + } + } + +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java new file mode 100644 index 0000000000000..096aafcffc09c --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertCommitActionExecutor.java @@ -0,0 +1,50 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class JavaInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { + + private List> inputRecords; + + public JavaInsertCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + List> inputRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT); + this.inputRecords = inputRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..349cf69dcc30b --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaInsertPreppedCommitActionExecutor.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class JavaInsertPreppedCommitActionExecutor> + extends BaseJavaCommitActionExecutor { + + private final List> preppedRecords; + + public JavaInsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return super.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java new file mode 100644 index 0000000000000..bd596bea541e1 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java @@ -0,0 +1,115 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; +import org.apache.hudi.exception.HoodieException; +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.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.BinaryEncoder; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +public class JavaMergeHelper extends AbstractMergeHelper>, + List, List> { + + private JavaMergeHelper() { + } + + private static class MergeHelperHolder { + private static final JavaMergeHelper JAVA_MERGE_HELPER = new JavaMergeHelper(); + } + + public static JavaMergeHelper newInstance() { + return JavaMergeHelper.MergeHelperHolder.JAVA_MERGE_HELPER; + } + + @Override + public void runMerge(HoodieTable>, List, List> table, + HoodieMergeHandle>, List, List> upsertHandle) throws IOException { + final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); + Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); + HoodieMergeHandle>, List, List> mergeHandle = upsertHandle; + HoodieBaseFile baseFile = mergeHandle.baseFileForMerge(); + + final GenericDatumWriter gWriter; + final GenericDatumReader gReader; + Schema readSchema; + if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) { + readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema(); + gWriter = new GenericDatumWriter<>(readSchema); + gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields()); + } else { + gReader = null; + gWriter = null; + readSchema = mergeHandle.getWriterSchemaWithMetafields(); + } + + BoundedInMemoryExecutor wrapper = null; + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + try { + final Iterator readerIterator; + if (baseFile.getBootstrapBaseFile().isPresent()) { + readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation); + } else { + readerIterator = reader.getRecordIterator(readSchema); + } + + ThreadLocal encoderCache = new ThreadLocal<>(); + ThreadLocal decoderCache = new ThreadLocal<>(); + wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), new IteratorBasedQueueProducer<>(readerIterator), + Option.of(new UpdateHandler(mergeHandle)), 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(); + } + mergeHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } + +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java new file mode 100644 index 0000000000000..06fce78a36d32 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertCommitActionExecutor.java @@ -0,0 +1,50 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class JavaUpsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { + + private List> inputRecords; + + public JavaUpsertCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + List> inputRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT); + this.inputRecords = inputRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return JavaWriteHelper.newInstance().write(instantTime, inputRecords, context, table, + config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java new file mode 100644 index 0000000000000..8eea5b5105826 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPreppedCommitActionExecutor.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hudi.table.action.HoodieWriteMetadata; + +import java.util.List; + +public class JavaUpsertPreppedCommitActionExecutor> + extends BaseJavaCommitActionExecutor { + + private final List> preppedRecords; + + public JavaUpsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List> preppedRecords) { + super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); + this.preppedRecords = preppedRecords; + } + + @Override + public HoodieWriteMetadata> execute() { + return super.execute(preppedRecords); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java new file mode 100644 index 0000000000000..ec7ea1641a442 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +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.util.collection.Pair; +import org.apache.hudi.index.HoodieIndex; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class JavaWriteHelper extends AbstractWriteHelper>, + List, List, R> { + + private JavaWriteHelper() { + } + + private static class WriteHelperHolder { + private static final JavaWriteHelper JAVA_WRITE_HELPER = new JavaWriteHelper(); + } + + public static JavaWriteHelper newInstance() { + return WriteHelperHolder.JAVA_WRITE_HELPER; + } + + @Override + public List> deduplicateRecords(List> records, + HoodieIndex>, List, List> index, + int parallelism) { + boolean isIndexingGlobal = index.isGlobal(); + Map>>> keyedRecords = records.stream().map(record -> { + HoodieKey hoodieKey = record.getKey(); + // If index used is global, then records are expected to differ in their partitionPath + Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey; + return Pair.of(key, record); + }).collect(Collectors.groupingBy(Pair::getLeft)); + + return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { + @SuppressWarnings("unchecked") + T reducedData = (T) rec1.getData().preCombine(rec2.getData()); + // we cannot allow the user to change the key or partitionPath, since that will affect + // everything + // so pick it from one of the records. + return new HoodieRecord(rec1.getKey(), reducedData); + }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); + } +} diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java new file mode 100644 index 0000000000000..4f192033dd333 --- /dev/null +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -0,0 +1,323 @@ +/* + * 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 org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.NumericUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Packs incoming records to be upserted, into buckets. + */ +public class UpsertPartitioner> implements Partitioner { + + private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class); + + /** + * List of all small files to be corrected. + */ + protected List smallFiles = new ArrayList<>(); + /** + * Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into. + */ + private int totalBuckets = 0; + /** + * Stat for the current workload. Helps in determining inserts, upserts etc. + */ + private WorkloadProfile profile; + /** + * Helps decide which bucket an incoming update should go to. + */ + private HashMap updateLocationToBucket; + /** + * Helps us pack inserts into 1 or more buckets depending on number of incoming records. + */ + private HashMap> partitionPathToInsertBucketInfos; + /** + * Remembers what type each bucket is for later. + */ + private HashMap bucketInfoMap; + + protected final HoodieTable table; + + protected final HoodieWriteConfig config; + + public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table, + HoodieWriteConfig config) { + updateLocationToBucket = new HashMap<>(); + partitionPathToInsertBucketInfos = new HashMap<>(); + bucketInfoMap = new HashMap<>(); + this.profile = profile; + this.table = table; + this.config = config; + assignUpdates(profile); + assignInserts(profile, context); + + LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n" + + "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n" + + "UpdateLocations mapped to buckets =>" + updateLocationToBucket); + } + + private void assignUpdates(WorkloadProfile profile) { + // each update location gets a partition + Set> partitionStatEntries = profile.getPartitionPathStatMap().entrySet(); + for (Map.Entry partitionStat : partitionStatEntries) { + for (Map.Entry> updateLocEntry : + partitionStat.getValue().getUpdateLocationToCount().entrySet()) { + addUpdateBucket(partitionStat.getKey(), updateLocEntry.getKey()); + } + } + } + + private int addUpdateBucket(String partitionPath, String fileIdHint) { + int bucket = totalBuckets; + updateLocationToBucket.put(fileIdHint, bucket); + BucketInfo bucketInfo = new BucketInfo(); + bucketInfo.bucketType = BucketType.UPDATE; + bucketInfo.fileIdPrefix = fileIdHint; + bucketInfo.partitionPath = partitionPath; + bucketInfoMap.put(totalBuckets, bucketInfo); + totalBuckets++; + return bucket; + } + + private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) { + // for new inserts, compute buckets depending on how many records we have for each partition + Set partitionPaths = profile.getPartitionPaths(); + long averageRecordSize = + averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(), + config); + LOG.info("AvgRecordSize => " + averageRecordSize); + + Map> partitionSmallFilesMap = + getSmallFilesForPartitions(new ArrayList(partitionPaths), context); + + for (String partitionPath : partitionPaths) { + WorkloadStat pStat = profile.getWorkloadStat(partitionPath); + if (pStat.getNumInserts() > 0) { + + List smallFiles = partitionSmallFilesMap.get(partitionPath); + this.smallFiles.addAll(smallFiles); + + LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles); + + long totalUnassignedInserts = pStat.getNumInserts(); + List bucketNumbers = new ArrayList<>(); + List recordsPerBucket = new ArrayList<>(); + + // first try packing this into one of the smallFiles + for (SmallFile smallFile : smallFiles) { + long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize, + totalUnassignedInserts); + if (recordsToAppend > 0) { + // create a new bucket or re-use an existing bucket + int bucket; + if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) { + bucket = updateLocationToBucket.get(smallFile.location.getFileId()); + LOG.info("Assigning " + recordsToAppend + " inserts to existing update bucket " + bucket); + } else { + bucket = addUpdateBucket(partitionPath, smallFile.location.getFileId()); + LOG.info("Assigning " + recordsToAppend + " inserts to new update bucket " + bucket); + } + bucketNumbers.add(bucket); + recordsPerBucket.add(recordsToAppend); + totalUnassignedInserts -= recordsToAppend; + } + } + + // if we have anything more, create new insert buckets, like normal + if (totalUnassignedInserts > 0) { + long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize(); + if (config.shouldAutoTuneInsertSplits()) { + insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize; + } + + int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket); + LOG.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts + + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket); + for (int b = 0; b < insertBuckets; b++) { + bucketNumbers.add(totalBuckets); + if (b < insertBuckets - 1) { + recordsPerBucket.add(insertRecordsPerBucket); + } else { + recordsPerBucket.add(totalUnassignedInserts - (insertBuckets - 1) * insertRecordsPerBucket); + } + BucketInfo bucketInfo = new BucketInfo(); + bucketInfo.bucketType = BucketType.INSERT; + bucketInfo.partitionPath = partitionPath; + bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx(); + bucketInfoMap.put(totalBuckets, bucketInfo); + totalBuckets++; + } + } + + // Go over all such buckets, and assign weights as per amount of incoming inserts. + List insertBuckets = new ArrayList<>(); + double curentCumulativeWeight = 0; + for (int i = 0; i < bucketNumbers.size(); i++) { + InsertBucket bkt = new InsertBucket(); + bkt.bucketNumber = bucketNumbers.get(i); + bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts(); + curentCumulativeWeight += bkt.weight; + insertBuckets.add(new InsertBucketCumulativeWeightPair(bkt, curentCumulativeWeight)); + } + LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets); + partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets); + } + } + } + + private Map> getSmallFilesForPartitions(List partitionPaths, HoodieEngineContext context) { + Map> partitionSmallFilesMap = new HashMap<>(); + if (partitionPaths != null && partitionPaths.size() > 0) { + context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions"); + partitionSmallFilesMap = context.mapToPair(partitionPaths, + partitionPath -> new ImmutablePair<>(partitionPath, getSmallFiles(partitionPath)), 0); + } + return partitionSmallFilesMap; + } + + /** + * Returns a list of small files in the given partition path. + */ + protected List getSmallFiles(String partitionPath) { + + // smallFiles only for partitionPath + List smallFileLocations = new ArrayList<>(); + + HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants(); + + if (!commitTimeline.empty()) { // if we have some commits + HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); + List allFiles = table.getBaseFileOnlyView() + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); + + for (HoodieBaseFile file : allFiles) { + if (file.getFileSize() < config.getParquetSmallFileLimit()) { + String filename = file.getFileName(); + SmallFile sf = new SmallFile(); + sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename)); + sf.sizeBytes = file.getFileSize(); + smallFileLocations.add(sf); + } + } + } + + return smallFileLocations; + } + + public BucketInfo getBucketInfo(int bucketNumber) { + return bucketInfoMap.get(bucketNumber); + } + + public List getInsertBuckets(String partitionPath) { + return partitionPathToInsertBucketInfos.get(partitionPath); + } + + @Override + public int getNumPartitions() { + return totalBuckets; + } + + @Override + public int getPartition(Object key) { + Pair> keyLocation = + (Pair>) key; + if (keyLocation.getRight().isPresent()) { + HoodieRecordLocation location = keyLocation.getRight().get(); + return updateLocationToBucket.get(location.getFileId()); + } else { + String partitionPath = keyLocation.getLeft().getPartitionPath(); + List targetBuckets = partitionPathToInsertBucketInfos.get(partitionPath); + // pick the target bucket to use based on the weights. + final long totalInserts = Math.max(1, profile.getWorkloadStat(partitionPath).getNumInserts()); + final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", keyLocation.getLeft().getRecordKey()); + final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts; + + int index = Collections.binarySearch(targetBuckets, new InsertBucketCumulativeWeightPair(new InsertBucket(), r)); + + if (index >= 0) { + return targetBuckets.get(index).getKey().bucketNumber; + } + + if ((-1 * index - 1) < targetBuckets.size()) { + return targetBuckets.get((-1 * index - 1)).getKey().bucketNumber; + } + + // return first one, by default + return targetBuckets.get(0).getKey().bucketNumber; + } + } + + /** + * Obtains the average record size based on records written during previous commits. Used for estimating how many + * records pack into one file. + */ + protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig) { + long avgSize = hoodieWriteConfig.getCopyOnWriteRecordSizeEstimate(); + long fileSizeThreshold = (long) (hoodieWriteConfig.getRecordSizeEstimationThreshold() * hoodieWriteConfig.getParquetSmallFileLimit()); + try { + if (!commitTimeline.empty()) { + // Go over the reverse ordered commits to get a more recent estimate of average record size. + Iterator instants = commitTimeline.getReverseOrderedInstants().iterator(); + while (instants.hasNext()) { + HoodieInstant instant = instants.next(); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + long totalBytesWritten = commitMetadata.fetchTotalBytesWritten(); + long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten(); + if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) { + avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten); + break; + } + } + } + } catch (Throwable t) { + // make this fail safe. + LOG.error("Error trying to compute average bytes/record ", t); + } + return avgSize; + } +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestHoodieJavaEngineContext.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestHoodieJavaEngineContext.java index b81c11b710f76..e67e78c019669 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestHoodieJavaEngineContext.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/common/TestHoodieJavaEngineContext.java @@ -20,11 +20,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hudi.DummyTaskContextSupplier; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Assertions; -import scala.Tuple2; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -77,7 +76,7 @@ public void testMapToPair() { Map resultMap = context.mapToPair(mapList, x -> { String[] splits = x.split("_"); - return Tuple2.apply(splits[0], splits[1]); + return new ImmutablePair<>(splits[0], splits[1]); }, 2); Assertions.assertNotNull(resultMap.get("hudi")); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestBase.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestBase.java new file mode 100644 index 0000000000000..171df5fe59356 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestBase.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.testutils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +import java.io.IOException; + +/** + * Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests. + */ +public class HoodieJavaClientTestBase extends HoodieJavaClientTestHarness { + + @BeforeEach + public void setUp() throws Exception { + initResources(); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + // Functional Interfaces for passing lambda and Hoodie Write API contexts + + @FunctionalInterface + public interface Function2 { + + R apply(T1 v1, T2 v2) throws IOException; + } +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java new file mode 100644 index 0000000000000..2b4c5d85c8f15 --- /dev/null +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java @@ -0,0 +1,242 @@ +/* + * 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.testutils; + +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; + +import java.io.IOException; +import java.io.Serializable; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +/** + * The test harness for resource initialization and cleanup. + */ +public abstract class HoodieJavaClientTestHarness extends HoodieCommonTestHarness implements Serializable { + + private static final Logger LOG = LogManager.getLogger(HoodieJavaClientTestHarness.class); + + private String testMethodName; + protected transient Configuration hadoopConf = null; + protected transient HoodieJavaEngineContext context = null; + protected transient TestJavaTaskContextSupplier taskContextSupplier = null; + protected transient FileSystem fs; + protected transient ExecutorService executorService; + protected transient HoodieTableMetaClient metaClient; + protected transient HoodieTableFileSystemView tableView; + protected transient HoodieJavaWriteClient writeClient; + + // dfs + protected String dfsBasePath; + protected transient HdfsTestService hdfsTestService; + protected transient MiniDFSCluster dfsCluster; + protected transient DistributedFileSystem dfs; + + @BeforeEach + public void setTestMethodName(TestInfo testInfo) { + if (testInfo.getTestMethod().isPresent()) { + testMethodName = testInfo.getTestMethod().get().getName(); + } else { + testMethodName = "Unknown"; + } + } + + /** + * Initializes resource group for the subclasses of {@link HoodieJavaClientTestHarness}. + */ + public void initResources() throws IOException { + initPath(); + hadoopConf = new Configuration(); + taskContextSupplier = new TestJavaTaskContextSupplier(); + context = new HoodieJavaEngineContext(hadoopConf, taskContextSupplier); + initTestDataGenerator(); + initFileSystem(); + initMetaClient(); + } + + public class TestJavaTaskContextSupplier extends TaskContextSupplier { + int partitionId = 0; + int stageId = 0; + long attemptId = 0; + + public void reset() { + stageId += 1; + } + + @Override + public Supplier getPartitionIdSupplier() { + return () -> partitionId; + } + + @Override + public Supplier getStageIdSupplier() { + return () -> stageId; + } + + @Override + public Supplier getAttemptIdSupplier() { + return () -> attemptId; + } + + @Override + public Option getProperty(EngineProperty prop) { + return Option.empty(); + } + } + + /** + * Cleanups resource group for the subclasses of {@link HoodieJavaClientTestHarness}. + */ + public void cleanupResources() throws IOException { + cleanupClients(); + cleanupTestDataGenerator(); + cleanupFileSystem(); + cleanupDFS(); + cleanupExecutorService(); + System.gc(); + } + + /** + * Initializes a file system with the hadoop configuration of Spark context. + */ + protected void initFileSystem() { + initFileSystemWithConfiguration(hadoopConf); + } + + /** + * Cleanups file system. + * + * @throws IOException + */ + protected void cleanupFileSystem() throws IOException { + if (fs != null) { + LOG.warn("Closing file-system instance used in previous test-run"); + fs.close(); + fs = null; + } + } + + /** + * Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by + * {@code getTableType()}. + * + * @throws IOException + */ + protected void initMetaClient() throws IOException { + initMetaClient(getTableType()); + } + + protected void initMetaClient(HoodieTableType tableType) throws IOException { + if (basePath == null) { + throw new IllegalStateException("The base path has not been initialized."); + } + + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); + } + + /** + * Cleanups hoodie clients. + */ + protected void cleanupClients() { + if (metaClient != null) { + metaClient = null; + } + if (writeClient != null) { + writeClient.close(); + writeClient = null; + } + if (tableView != null) { + tableView.close(); + tableView = null; + } + } + + /** + * Cleanups the distributed file system. + * + * @throws IOException + */ + protected void cleanupDFS() throws IOException { + if (hdfsTestService != null) { + hdfsTestService.stop(); + dfsCluster.shutdown(); + hdfsTestService = null; + dfsCluster = null; + dfs = null; + } + // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the + // same JVM + FileSystem.closeAll(); + } + + /** + * Cleanups the executor service. + */ + protected void cleanupExecutorService() { + if (this.executorService != null) { + this.executorService.shutdownNow(); + this.executorService = null; + } + } + + private void initFileSystemWithConfiguration(Configuration configuration) { + if (basePath == null) { + throw new IllegalStateException("The base path has not been initialized."); + } + + fs = FSUtils.getFs(basePath, configuration); + if (fs instanceof LocalFileSystem) { + LocalFileSystem lfs = (LocalFileSystem) fs; + // With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream + // This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open + // So, for the tests, we enforce checksum verification to circumvent the problem + lfs.setVerifyChecksum(true); + } + } + + public HoodieJavaWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) { + if (null != writeClient) { + writeClient.close(); + writeClient = null; + } + writeClient = new HoodieJavaWriteClient(context, cfg); + return writeClient; + } +} diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 5cc6ad6560b6c..20f0565976468 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -15,17 +15,16 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi-client org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 hudi-spark-client - ${parent.version} + 0.8.0-SNAPSHOT hudi-spark-client jar diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java index 152a901a77f60..5235a3cd0a15d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java @@ -21,7 +21,7 @@ import org.apache.hudi.client.AbstractCompactor; import org.apache.hudi.client.AbstractHoodieWriteClient; import org.apache.hudi.client.HoodieSparkCompactor; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; public class SparkAsyncCompactService extends AsyncCompactService { @@ -31,6 +31,6 @@ public SparkAsyncCompactService(HoodieEngineContext context, AbstractHoodieWrite @Override protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) { - return new HoodieSparkCompactor(client); + return new HoodieSparkCompactor(client, this.context); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java index b81570de9ef10..60a064ab32bf9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java @@ -18,6 +18,7 @@ package org.apache.hudi.client; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -33,16 +34,20 @@ public class HoodieSparkCompactor extends AbstractCompactor>, JavaRDD, JavaRDD> { private static final Logger LOG = LogManager.getLogger(HoodieSparkCompactor.class); + private transient HoodieEngineContext context; - public HoodieSparkCompactor(AbstractHoodieWriteClient>, JavaRDD, JavaRDD> compactionClient) { + public HoodieSparkCompactor(AbstractHoodieWriteClient>, JavaRDD, JavaRDD> compactionClient, + HoodieEngineContext context) { super(compactionClient); + this.context = context; } @Override public void compact(HoodieInstant instant) throws IOException { LOG.info("Compactor executing compaction " + instant); - SparkRDDWriteClient writeClient = (SparkRDDWriteClient)compactionClient; + SparkRDDWriteClient writeClient = (SparkRDDWriteClient) compactionClient; JavaRDD res = writeClient.compact(instant.getTimestamp()); + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status"); long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count(); if (numWriteErrors != 0) { // We treat even a single error in compaction as fatal diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 56f06898abba2..6abefbd276f48 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -18,13 +18,18 @@ package org.apache.hudi.client; -import org.apache.hudi.client.common.HoodieEngineContext; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.HoodieCommitMetadata; 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.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -34,24 +39,28 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metrics.DistributedRegistry; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.compact.SparkCompactHelpers; import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; - -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.List; import java.util.Map; @@ -133,8 +142,7 @@ public JavaRDD upsert(JavaRDD> records, String inst HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime); table.validateUpsertSchema(); - setOperationType(WriteOperationType.UPSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.UPSERT); HoodieWriteMetadata> result = table.upsert(context, instantTime, records); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); @@ -147,8 +155,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime); table.validateUpsertSchema(); - setOperationType(WriteOperationType.UPSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.UPSERT_PREPPED); HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -158,8 +165,7 @@ public JavaRDD insert(JavaRDD> records, String inst HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime); table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.INSERT); HoodieWriteMetadata> result = table.insert(context,instantTime, records); return postWrite(result, instantTime, table); } @@ -169,8 +175,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime); table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.INSERT_PREPPED); HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); return postWrite(result, instantTime, table); } @@ -185,12 +190,27 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); table.validateInsertSchema(); - setOperationType(WriteOperationType.INSERT_OVERWRITE); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE); HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); } + + /** + * Removes all existing records of the Hoodie table and inserts the given HoodieRecords, into the table. + + * @param records HoodieRecords to insert + * @param instantTime Instant time of the commit + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public HoodieWriteResult insertOverwriteTable(JavaRDD> records, final String instantTime) { + HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime); + table.validateInsertSchema(); + preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE); + HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records); + return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + } + @Override public JavaRDD bulkInsert(JavaRDD> records, String instantTime) { return bulkInsert(records, instantTime, Option.empty()); @@ -201,8 +221,7 @@ public JavaRDD bulkInsert(JavaRDD> records, String HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime); table.validateInsertSchema(); - setOperationType(WriteOperationType.BULK_INSERT); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.BULK_INSERT); HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -212,8 +231,7 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime); table.validateInsertSchema(); - setOperationType(WriteOperationType.BULK_INSERT_PREPPED); - this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED); HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); return postWrite(result, instantTime, table); } @@ -221,11 +239,18 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr @Override public JavaRDD delete(JavaRDD keys, String instantTime) { HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); - setOperationType(WriteOperationType.DELETE); + preWrite(instantTime, WriteOperationType.DELETE); HoodieWriteMetadata> result = table.delete(context,instantTime, keys); return postWrite(result, instantTime, table); } + public HoodieWriteResult deletePartitions(List partitions, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime); + preWrite(instantTime, WriteOperationType.DELETE_PARTITION); + HoodieWriteMetadata> result = table.deletePartitions(context,instantTime, partitions); + return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + } + @Override protected JavaRDD postWrite(HoodieWriteMetadata> result, String instantTime, @@ -260,6 +285,7 @@ public void commitCompaction(String compactionInstantTime, JavaRDD protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD writeStatuses, HoodieTable>, JavaRDD, JavaRDD> table, String compactionCommitTime) { + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); finalizeWrite(table, compactionCommitTime, writeStats); LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata); @@ -296,6 +322,57 @@ protected JavaRDD compact(String compactionInstantTime, boolean sho return statuses; } + @Override + public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant); + if (pendingClusteringTimeline.containsInstant(inflightInstant)) { + rollbackInflightClustering(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + clusteringTimer = metrics.getClusteringCtx(); + LOG.info("Starting clustering at " + clusteringInstant); + HoodieWriteMetadata> clusteringMetadata = table.cluster(context, clusteringInstant); + JavaRDD statuses = clusteringMetadata.getWriteStatuses(); + if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) { + completeClustering((HoodieReplaceCommitMetadata) clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant); + } + return clusteringMetadata; + } + + protected void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD writeStatuses, + HoodieTable>, JavaRDD, JavaRDD> table, + String clusteringCommitTime) { + + List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + if (!writeStatuses.filter(WriteStatus::hasErrors).isEmpty()) { + throw new HoodieClusteringException("Clustering failed to write to files:" + + writeStatuses.filter(WriteStatus::hasErrors).map(WriteStatus::getFileId).collect()); + } + finalizeWrite(table, clusteringCommitTime, writeStats); + try { + LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata); + table.getActiveTimeline().transitionReplaceInflightToComplete( + HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); + } + + if (clusteringTimer != null) { + long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(clusteringCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " + + config.getBasePath() + " at time " + clusteringCommitTime, e); + } + } + LOG.info("Clustering successfully on commit " + clusteringCommitTime); + } + @Override protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); @@ -316,4 +393,38 @@ private HoodieTable>, JavaRDD, JavaRDD> + extends PartitionAwareClusteringPlanStrategy>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(SparkRecentDaysClusteringPlanStrategy.class); + + public SparkRecentDaysClusteringPlanStrategy(HoodieSparkCopyOnWriteTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public SparkRecentDaysClusteringPlanStrategy(HoodieSparkMergeOnReadTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + List, Integer>> fileSliceGroups = new ArrayList<>(); + List currentGroup = new ArrayList<>(); + int totalSizeSoFar = 0; + for (FileSlice currentSlice : fileSlices) { + // assume each filegroup size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : getWriteConfig().getParquetMaxFileSize(); + // check if max size is reached and create new group, if needed. + if (totalSizeSoFar >= getWriteConfig().getClusteringMaxBytesInGroup() && !currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, getNumberOfOutputFileGroups(totalSizeSoFar, getWriteConfig().getClusteringTargetFileMaxBytes()))); + currentGroup = new ArrayList<>(); + totalSizeSoFar = 0; + } + currentGroup.add(currentSlice); + } + if (!currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, getNumberOfOutputFileGroups(totalSizeSoFar, getWriteConfig().getClusteringTargetFileMaxBytes()))); + } + + return fileSliceGroups.stream().map(fileSliceGroup -> HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(CLUSTERING_SORT_COLUMNS_PROPERTY, getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering(); + return partitionPaths.stream() + .sorted(Comparator.reverseOrder()) + .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size()) + .collect(Collectors.toList()); + } + + @Override + protected Stream getFileSlicesEligibleForClustering(final String partition) { + return super.getFileSlicesEligibleForClustering(partition) + // Only files that have basefile size smaller than small file size are eligible. + .filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit()); + } + + private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) { + return (int) Math.ceil(groupSize / (double) targetFileSize); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java new file mode 100644 index 0000000000000..cfbc2ec22d319 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -0,0 +1,89 @@ +/* + * 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.clustering.run.strategy; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +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.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY; + +/** + * Clustering Strategy based on following. + * 1) Spark execution engine. + * 2) Uses bulk_insert to write data into new files. + */ +public class SparkSortAndSizeExecutionStrategy> + extends ClusteringExecutionStrategy>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(SparkSortAndSizeExecutionStrategy.class); + + public SparkSortAndSizeExecutionStrategy(HoodieSparkCopyOnWriteTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public SparkSortAndSizeExecutionStrategy(HoodieSparkMergeOnReadTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public JavaRDD performClustering(final JavaRDD> inputRecords, final int numOutputGroups, + final String instantTime, final Map strategyParams, final Schema schema) { + Properties props = getWriteConfig().getProps(); + props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM, String.valueOf(numOutputGroups)); + // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files. + props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, Boolean.FALSE.toString()); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build(); + return (JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + false, getPartitioner(strategyParams, schema), true, numOutputGroups); + } + + /** + * Create BulkInsertPartitioner based on strategy params. + */ + protected Option> getPartitioner(Map strategyParams, Schema schema) { + if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY)) { + return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY).split(","), + HoodieAvroUtils.addMetadataFields(schema))); + } else { + return Option.empty(); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java new file mode 100644 index 0000000000000..134e490246680 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.update.strategy; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieClusteringUpdateException; +import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.HashSet; +import java.util.List; + +/** + * Update strategy based on following. + * if some file group have update record, throw exception + */ +public class SparkRejectUpdateStrategy> extends UpdateStrategy>> { + private static final Logger LOG = LogManager.getLogger(SparkRejectUpdateStrategy.class); + + public SparkRejectUpdateStrategy(HoodieSparkEngineContext engineContext, HashSet fileGroupsInPendingClustering) { + super(engineContext, fileGroupsInPendingClustering); + } + + private List getGroupIdsWithUpdate(JavaRDD> inputRecords) { + List fileGroupIdsWithUpdates = inputRecords + .filter(record -> record.getCurrentLocation() != null) + .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collect(); + return fileGroupIdsWithUpdates; + } + + @Override + public JavaRDD> handleUpdate(JavaRDD> taggedRecordsRDD) { + List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); + fileGroupIdsWithRecordUpdate.forEach(fileGroupIdWithRecordUpdate -> { + if (fileGroupsInPendingClustering.contains(fileGroupIdWithRecordUpdate)) { + String msg = String.format("Not allowed to update the clustering file group %s. " + + "For pending clustering operations, we are not going to support update for now.", + fileGroupIdWithRecordUpdate.toString()); + LOG.error(msg); + throw new HoodieClusteringUpdateException(msg); + } + }); + return taggedRecordsRDD; + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java index 0f17511b56d99..d869ec77a720b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java @@ -20,14 +20,18 @@ import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.common.config.SerializableConfiguration; -import org.apache.hudi.client.common.function.SerializableConsumer; -import org.apache.hudi.client.common.function.SerializableFunction; -import org.apache.hudi.client.common.function.SerializablePairFunction; +import org.apache.hudi.common.engine.EngineProperty; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.function.SerializableConsumer; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; +import scala.Tuple2; import java.util.List; import java.util.Map; @@ -82,9 +86,15 @@ public void foreach(List data, SerializableConsumer consumer, int para @Override public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { if (Objects.nonNull(parallelism)) { - return javaSparkContext.parallelize(data, parallelism).mapToPair(func::call).collectAsMap(); + return javaSparkContext.parallelize(data, parallelism).mapToPair(input -> { + Pair pair = func.call(input); + return new Tuple2(pair.getLeft(), pair.getRight()); + }).collectAsMap(); } else { - return javaSparkContext.parallelize(data).mapToPair(func::call).collectAsMap(); + return javaSparkContext.parallelize(data).mapToPair(input -> { + Pair pair = func.call(input); + return new Tuple2(pair.getLeft(), pair.getRight()); + }).collectAsMap(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkRowDeserializer.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkRowDeserializer.java new file mode 100644 index 0000000000000..66b8b78b56920 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkRowDeserializer.java @@ -0,0 +1,28 @@ +/* + * 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 org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; + +import java.io.Serializable; + +public interface SparkRowDeserializer extends Serializable { + Row deserializeRow(InternalRow internalRow); +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index ec90ef88ed86f..bf9f90a0c948f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -19,8 +19,9 @@ package org.apache.hudi.execution; import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.TaskContextSupplier; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; @@ -34,14 +35,18 @@ public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { + private boolean useWriterSchema; + public SparkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String idPrefix, - TaskContextSupplier taskContextSupplier) { + TaskContextSupplier taskContextSupplier, + boolean useWriterSchema) { super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); + this.useWriterSchema = useWriterSchema; } public SparkLazyInsertIterable(Iterator> recordItr, @@ -53,6 +58,7 @@ public SparkLazyInsertIterable(Iterator> recordItr, TaskContextSupplier taskContextSupplier, WriteHandleFactory writeHandleFactory) { super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory); + this.useWriterSchema = false; } @Override @@ -61,7 +67,10 @@ protected List computeNext() { BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = null; try { - final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + if (useWriterSchema) { + schema = HoodieAvroUtils.addMetadataFields(schema); + } bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema)); final List result = bufferedIteratorExecutor.execute(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java index db73a9c3e7e70..96da3969eecb2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java @@ -41,20 +41,22 @@ public class BulkInsertMapFunction private HoodieWriteConfig config; private HoodieTable hoodieTable; private List fileIDPrefixes; + private boolean useWriterSchema; public BulkInsertMapFunction(String instantTime, boolean areRecordsSorted, HoodieWriteConfig config, HoodieTable hoodieTable, - List fileIDPrefixes) { + List fileIDPrefixes, boolean useWriterSchema) { this.instantTime = instantTime; this.areRecordsSorted = areRecordsSorted; this.config = config; this.hoodieTable = hoodieTable; this.fileIDPrefixes = fileIDPrefixes; + this.useWriterSchema = useWriterSchema; } @Override public Iterator> call(Integer partition, Iterator> recordItr) { return new SparkLazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable, - fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier()); + fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier(), useWriterSchema); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java new file mode 100644 index 0000000000000..209531dd2e090 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.execution.bulkinsert; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; + +/** + * A partitioner that does sorting based on specified column values for each RDD partition. + * + * @param HoodieRecordPayload type + */ +public class RDDCustomColumnsSortPartitioner + implements BulkInsertPartitioner>> { + + private final String[] sortColumnNames; + private final SerializableSchema serializableSchema; + + public RDDCustomColumnsSortPartitioner(String[] columnNames, Schema schema) { + this.sortColumnNames = columnNames; + this.serializableSchema = new SerializableSchema(schema); + } + + @Override + public JavaRDD> repartitionRecords(JavaRDD> records, + int outputSparkPartitions) { + final String[] sortColumns = this.sortColumnNames; + final SerializableSchema schema = this.serializableSchema; + return records.sortBy(record -> getRecordSortColumnValues(record, sortColumns, schema), + true, outputSparkPartitions); + } + + @Override + public boolean arePartitionRecordsSorted() { + return true; + } + + private static Object getRecordSortColumnValues(HoodieRecord record, + String[] sortColumns, + SerializableSchema schema) { + try { + GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema.get()).get(); + if (sortColumns.length == 1) { + return HoodieAvroUtils.getNestedFieldVal(genericRecord, sortColumns[0], true); + } else { + StringBuilder sb = new StringBuilder(); + for (String col : sortColumns) { + sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true)); + } + + return sb.toString(); + } + } catch (IOException e) { + throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java index dd73bf2e76568..085c0d9b1cc68 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java @@ -21,7 +21,7 @@ import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java index 55ce8d2cc90c0..0cd839aa87be3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java @@ -19,7 +19,7 @@ package org.apache.hudi.index; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java index 894b41b51c6bf..2dd485ebc6a05 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java @@ -20,7 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -122,13 +122,15 @@ private JavaPairRDD lookupIndex( // Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, // that contains it. + JavaRDD> fileComparisonsRDD = + explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD); Map comparisonsPerFileGroup = - computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD); + computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); int inputParallelism = partitionRecordKeyPairRDD.partitions().size(); int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${" + config.getBloomIndexParallelism() + "}"); - return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, joinParallelism, hoodieTable, + return findMatchingFilesForRecordKeys(fileComparisonsRDD, joinParallelism, hoodieTable, comparisonsPerFileGroup); } @@ -137,14 +139,14 @@ private JavaPairRDD lookupIndex( */ private Map computeComparisonsPerFileGroup(final Map recordsPerPartition, final Map> partitionToFileInfo, - JavaPairRDD partitionRecordKeyPairRDD) { - + final JavaRDD> fileComparisonsRDD, + final HoodieEngineContext context) { Map fileToComparisons; if (config.getBloomIndexPruneByRanges()) { // we will just try exploding the input and then count to determine comparisons // FIX(vc): Only do sampling here and extrapolate? - fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD) - .mapToPair(t -> t).countByKey(); + context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files"); + fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey(); } else { fileToComparisons = new HashMap<>(); partitionToFileInfo.forEach((key, value) -> { @@ -252,11 +254,10 @@ JavaRDD> explodeRecordRDDWithFileComparisons( * Make sure the parallelism is atleast the groupby parallelism for tagging location */ JavaPairRDD findMatchingFilesForRecordKeys( - final Map> partitionToFileIndexInfo, - JavaPairRDD partitionRecordKeyPairRDD, int shuffleParallelism, HoodieTable hoodieTable, + JavaRDD> fileComparisonsRDD, + int shuffleParallelism, + HoodieTable hoodieTable, Map fileGroupToComparisons) { - JavaRDD> fileComparisonsRDD = - explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD); if (config.useBloomIndexBucketizedChecking()) { Partitioner partitioner = new BucketizedBloomCheckPartitioner(shuffleParallelism, fileGroupToComparisons, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java index 771c01ab875de..21b9d40601a78 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java @@ -18,7 +18,7 @@ package org.apache.hudi.index.bloom; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; @@ -28,7 +28,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieTable; @@ -36,7 +35,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.Optional; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -62,13 +60,8 @@ public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) { List> loadInvolvedFiles(List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - try { - List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - config.shouldAssumeDatePartitioning()); - return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); - } catch (IOException e) { - throw new HoodieIOException("Failed to load all partitions", e); - } + List allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); + return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index 5b67f838509bd..22ee65cc302f5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -19,9 +19,9 @@ package org.apache.hudi.index.hbase; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.utils.SparkMemoryUtils; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java index bdb4991cf76e3..63e13c463ab44 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java @@ -19,7 +19,7 @@ package org.apache.hudi.index.simple; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieBaseFile; @@ -31,14 +31,12 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.table.HoodieTable; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import scala.Tuple2; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -103,13 +101,9 @@ protected JavaPairRDD fetchAllRecordLocations(H protected List> getAllBaseFilesInTable(final HoodieEngineContext context, final HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - try { - List allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning()); - // Obtain the latest data files from all the partitions. - return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable); - } catch (IOException e) { - throw new HoodieIOException("Failed to load all partitions", e); - } + List allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); + // Obtain the latest data files from all the partitions. + return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java index 3f167e2ebdd85..ab6e2586749e5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java @@ -20,8 +20,8 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -147,6 +147,7 @@ protected JavaPairRDD fetchRecordLocations(Hood JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism)); return jsc.parallelize(baseFiles, fetchParallelism) - .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations()); + .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile) + .locations().map(x -> Tuple2.apply(((Pair)x).getLeft(), ((Pair)x).getRight())).iterator()); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java index 6727b79d78477..77896d2e88a06 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java @@ -113,7 +113,7 @@ private String getPartitionPath(Option record, Option row) { partitionPath.append(new TimestampBasedKeyGenerator(config, partitionPathField).getPartitionPath(row.get())); } } catch (IOException ioe) { - throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class"); + throw new HoodieKeyGeneratorException("Unable to initialise TimestampBasedKeyGenerator class", ioe); } break; default: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java new file mode 100644 index 0000000000000..7c12a9e001024 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.metrics.DistributedRegistry; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.conf.Configuration; +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.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { + + private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class); + + public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) { + return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context); + } + + SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { + super(hadoopConf, writeConfig, engineContext); + } + + @Override + protected void initRegistry() { + if (metadataWriteConfig.isMetricsOn()) { + Registry registry; + if (metadataWriteConfig.isExecutorMetricsEnabled()) { + registry = Registry.getRegistry("HoodieMetadata", DistributedRegistry.class.getName()); + } else { + registry = Registry.getRegistry("HoodieMetadata"); + } + this.metrics = Option.of(new HoodieMetadataMetrics(registry)); + } else { + this.metrics = Option.empty(); + } + } + + @Override + protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) { + try { + metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> { + if (registry instanceof DistributedRegistry) { + HoodieSparkEngineContext sparkEngineContext = (HoodieSparkEngineContext) engineContext; + ((DistributedRegistry) registry).register(sparkEngineContext.getJavaSparkContext()); + } + }); + + if (enabled) { + bootstrapIfNeeded(engineContext, datasetMetaClient); + } + } catch (IOException e) { + LOG.error("Failed to initialize metadata table. Disabling the writer.", e); + enabled = false; + } + } + + @Override + protected void commit(List records, String partitionName, String instantTime) { + ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled"); + JavaRDD recordRDD = prepRecords(records, partitionName); + + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { + writeClient.startCommitWithTime(instantTime); + List statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect(); + statuses.forEach(writeStatus -> { + if (writeStatus.hasErrors()) { + throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); + } + }); + // trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future + // delta commits synced over will not have an instant time lesser than the last completed instant on the + // metadata table. + if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) { + writeClient.compact(instantTime + "001"); + } + writeClient.clean(instantTime + "002"); + } + + // Update total size of the metadata and count of base/log files + metrics.ifPresent(m -> { + try { + Map stats = m.getStats(false, metaClient, metadata); + m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)), + Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)), + Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)), + Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES))); + } catch (HoodieIOException e) { + LOG.error("Could not publish metadata size metrics", e); + } + }); + } + + /** + * Tag each record with the location. + * + * Since we only read the latest base file in a partition, we tag the records with the instant time of the latest + * base file. + */ + private JavaRDD prepRecords(List records, String partitionName) { + HoodieTable table = HoodieSparkTable.create(metadataWriteConfig, engineContext); + TableFileSystemView.SliceView fsView = table.getSliceView(); + List baseFiles = fsView.getLatestFileSlices(partitionName) + .map(FileSlice::getBaseFile) + .filter(Option::isPresent) + .map(Option::get) + .collect(Collectors.toList()); + + // All the metadata fits within a single base file + if (partitionName.equals(MetadataPartitionType.FILES.partitionPath())) { + if (baseFiles.size() > 1) { + throw new HoodieMetadataException("Multiple base files found in metadata partition"); + } + } + + JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext(); + String fileId; + String instantTime; + if (!baseFiles.isEmpty()) { + fileId = baseFiles.get(0).getFileId(); + instantTime = baseFiles.get(0).getCommitTime(); + } else { + // If there is a log file then we can assume that it has the data + List logFiles = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()) + .map(FileSlice::getLatestLogFile) + .filter(Option::isPresent) + .map(Option::get) + .collect(Collectors.toList()); + if (logFiles.isEmpty()) { + // No base and log files. All are new inserts + return jsc.parallelize(records, 1); + } + + fileId = logFiles.get(0).getFileId(); + instantTime = logFiles.get(0).getBaseCommitTime(); + } + + return jsc.parallelize(records, 1).map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId))); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java new file mode 100644 index 0000000000000..22b3afd10e058 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metrics/DistributedRegistry.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metrics; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hudi.common.metrics.Registry; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.util.AccumulatorV2; + +/** + * Lightweight Metrics Registry to track Hudi events. + */ +public class DistributedRegistry extends AccumulatorV2, Map> + implements Registry, Serializable { + private String name; + ConcurrentHashMap counters = new ConcurrentHashMap<>(); + + public DistributedRegistry(String name) { + this.name = name; + } + + public void register(JavaSparkContext jsc) { + if (!isRegistered()) { + jsc.sc().register(this); + } + } + + @Override + public void clear() { + counters.clear(); + } + + @Override + public void increment(String name) { + counters.merge(name, 1L, (oldValue, newValue) -> oldValue + newValue); + } + + @Override + public void add(String name, long value) { + counters.merge(name, value, (oldValue, newValue) -> oldValue + newValue); + } + + /** + * Get all Counter type metrics. + */ + @Override + public Map getAllCounts(boolean prefixWithRegistryName) { + HashMap countersMap = new HashMap<>(); + counters.forEach((k, v) -> { + String key = prefixWithRegistryName ? name + "." + k : k; + countersMap.put(key, v); + }); + return countersMap; + } + + @Override + public void add(Map arg) { + arg.forEach((key, value) -> add(key, value)); + } + + @Override + public AccumulatorV2, Map> copy() { + DistributedRegistry registry = new DistributedRegistry(name); + counters.forEach((key, value) -> registry.add(key, value)); + return registry; + } + + @Override + public boolean isZero() { + return counters.isEmpty(); + } + + @Override + public void merge(AccumulatorV2, Map> acc) { + acc.value().forEach((key, value) -> add(key, value)); + } + + @Override + public void reset() { + counters.clear(); + } + + @Override + public Map value() { + return counters; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index f2b336432b247..12867e636cf03 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -19,13 +19,14 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -44,11 +45,15 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; import org.apache.hudi.table.action.clean.SparkCleanActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; +import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; +import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkDeletePartitionCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkMergeHelper; import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor; @@ -56,7 +61,6 @@ import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -105,6 +109,11 @@ public HoodieWriteMetadata> delete(HoodieEngineContext cont return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); } + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + return new SparkDeletePartitionCommitActionExecutor(context, config, this, instantTime, partitions).execute(); + } + @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { @@ -129,6 +138,11 @@ public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String i return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute(); } + @Override + public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, JavaRDD> records) { + return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute(); + } + @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); @@ -139,6 +153,19 @@ public HoodieWriteMetadata> compact(HoodieEngineContext con throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } + @Override + public Option scheduleClustering(HoodieEngineContext context, + String instantTime, + Option> extraMetadata) { + return new SparkClusteringPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); + } + + @Override + public HoodieWriteMetadata> cluster(HoodieEngineContext context, + String clusteringInstantTime) { + return new SparkExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); + } + @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); @@ -156,7 +183,7 @@ public Iterator> handleUpdate(String instantTime, String parti return handleUpdateInternal(upsertHandle, instantTime, fileId); } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime, + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime, String fileId) throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException( @@ -166,11 +193,12 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups } // TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + if (upsertHandle.getPartitionPath() == null) { LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); + + upsertHandle.writeStatuses()); } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); + + return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); } protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId, @@ -186,10 +214,10 @@ protected HoodieMergeHandle getUpdateHandle(String instantTime, String partition public Iterator> handleInsert(String instantTime, String partitionPath, String fileId, Map> recordMap) { - HoodieCreateHandle createHandle = + HoodieCreateHandle createHandle = new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier); createHandle.write(); - return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator(); + return Collections.singletonList(createHandle.close()).iterator(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java index 0a60dcc50f032..997116ec06448 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java @@ -22,8 +22,8 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index d5fb4ee018a69..dd8106f52a6ea 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -19,8 +19,8 @@ package org.apache.hudi.table; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 2dd9fd039dbea..1aca28b6d8f79 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -291,7 +291,7 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) { Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); - HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); Schema avroSchema = null; try { @@ -329,7 +329,8 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } - BootstrapWriteStatus writeStatus = (BootstrapWriteStatus)bootstrapHandle.getWriteStatus(); + + BootstrapWriteStatus writeStatus = (BootstrapWriteStatus) bootstrapHandle.writeStatuses().get(0); BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping( config.getBootstrapSourceBasePath(), srcPartitionPath, partitionPath, srcFileStatus, writeStatus.getFileId()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java index bbd5c1fb0e4cc..ba2d42f434861 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java @@ -23,9 +23,9 @@ import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.common.HoodieCleanStat; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieCleanStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java new file mode 100644 index 0000000000000..1f71aa4dfac7c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java @@ -0,0 +1,74 @@ +/* + * 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.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieEngineContext; +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.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkClusteringPlanActionExecutor extends + BaseClusteringPlanActionExecutor>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanActionExecutor.class); + + public SparkClusteringPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected Option createClusteringPlan() { + LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); + Option lastClusteringInstant = table.getActiveTimeline().getCompletedReplaceTimeline().lastInstant(); + + int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) + .countInstants(); + if (config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getInlineClusterMaxCommits()); + return Option.empty(); + } + + LOG.info("Generating clustering plan for table " + config.getBasePath()); + ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) + ReflectionUtils.loadClass(config.getClusteringPlanStrategyClass(), table, context, config); + return strategy.generateClusteringPlan(); + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java new file mode 100644 index 0000000000000..1c8d9022a3e4c --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -0,0 +1,230 @@ +/* + * 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.cluster; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieCommitMetadata; +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.WriteOperationType; +import org.apache.hudi.common.table.log.HoodieFileSliceReader; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.io.IOUtils; +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.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; +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.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +public class SparkExecuteClusteringCommitActionExecutor> + extends BaseSparkCommitActionExecutor { + + private static final Logger LOG = LogManager.getLogger(SparkExecuteClusteringCommitActionExecutor.class); + private final HoodieClusteringPlan clusteringPlan; + + public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime) { + super(context, config, table, instantTime, WriteOperationType.CLUSTER); + this.clusteringPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)) + .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException("Unable to read clustering plan for instant: " + instantTime)); + } + + @Override + public HoodieWriteMetadata> execute() { + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime); + // Mark instant as clustering inflight + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + table.getMetaClient().reloadActiveTimeline(); + + JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(context); + // execute clustering for each group async and collect WriteStatus + JavaRDD writeStatusRDD = clusteringPlan.getInputGroups().stream() + .map(inputGroup -> runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams())) + .map(CompletableFuture::join) + .reduce((rdd1, rdd2) -> rdd1.union(rdd2)).orElse(engineContext.emptyRDD()); + if (writeStatusRDD.isEmpty()) { + throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + " #groups: " + clusteringPlan.getInputGroups().size()); + } + + HoodieWriteMetadata> writeMetadata = buildWriteMetadata(writeStatusRDD); + updateIndexAndCommitIfNeeded(writeStatusRDD, writeMetadata); + if (!writeMetadata.getCommitMetadata().isPresent()) { + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStatusRDD.map(WriteStatus::getStat).collect(), writeMetadata.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + writeMetadata.setCommitMetadata(Option.of(commitMetadata)); + } + return writeMetadata; + } + + /** + * Submit job to execute clustering for the group. + */ + private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams) { + CompletableFuture> writeStatusesFuture = CompletableFuture.supplyAsync(() -> { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + JavaRDD> inputRecords = readRecordsForGroup(jsc, clusteringGroup); + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + return ((ClusteringExecutionStrategy>, JavaRDD, JavaRDD>) + ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(), table, context, config)) + .performClustering(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema); + }); + + return writeStatusesFuture; + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan).collect( + Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList()))); + } + + /** + * Get RDD of all records for the group. This includes all records from file slice (Apply updates from log files, if any). + */ + private JavaRDD> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup) { + List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); + boolean hasLogFiles = clusteringOps.stream().filter(op -> op.getDeltaFilePaths().size() > 0).findAny().isPresent(); + if (hasLogFiles) { + // if there are log files, we read all records into memory for a file group and apply updates. + return readRecordsForGroupWithLogs(jsc, clusteringOps); + } else { + // We want to optimize reading records for case there are no log files. + return readRecordsForGroupBaseFiles(jsc, clusteringOps); + } + } + + /** + * Read records from baseFiles, apply updates and convert to RDD. + */ + private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext jsc, + List clusteringOps) { + return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + List>> recordIterators = new ArrayList<>(); + clusteringOpsPartition.forEachRemaining(clusteringOp -> { + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps()); + LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(table.getMetaClient().getFs(), + table.getMetaClient().getBasePath(), clusteringOp.getDeltaFilePaths(), readerSchema, instantTime, + maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(), + config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), + config.getSpillableMapBasePath()); + + recordIterators.add(HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema, + table.getMetaClient().getTableConfig().getPayloadClass())); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + + return new ConcatenatingIterator<>(recordIterators); + }); + } + + /** + * Read records from baseFiles and convert to RDD. + */ + private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext jsc, + List clusteringOps) { + return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + List> iteratorsForPartition = new ArrayList<>(); + clusteringOpsPartition.forEachRemaining(clusteringOp -> { + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema)); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + + return new ConcatenatingIterator<>(iteratorsForPartition); + }).map(this::transform); + } + + /** + * Transform IndexedRecord into HoodieRecord. + */ + private HoodieRecord transform(IndexedRecord indexedRecord) { + GenericRecord record = (GenericRecord) indexedRecord; + String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieKey hoodieKey = new HoodieKey(key, partition); + + HoodieRecordPayload avroPayload = ReflectionUtils.loadPayload(table.getMetaClient().getTableConfig().getPayloadClass(), + new Object[] {Option.of(record)}, Option.class); + HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload); + return hoodieRecord; + } + + private HoodieWriteMetadata> buildWriteMetadata(JavaRDD writeStatusJavaRDD) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeStatusJavaRDD)); + result.setWriteStatuses(writeStatusJavaRDD); + result.setWriteStats(writeStatusJavaRDD.map(WriteStatus::getStat).collect()); + result.setCommitMetadata(Option.empty()); + result.setCommitted(false); + return result; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index ad62db9250eda..2fabbbfbc9be6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -20,7 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.SparkMemoryUtils; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -28,11 +28,13 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -46,6 +48,7 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; @@ -59,11 +62,13 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; +import java.util.stream.Collectors; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Set; import java.util.Map; public abstract class BaseSparkCommitActionExecutor extends @@ -88,6 +93,18 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, super(context, config, table, instantTime, operationType, extraMetadata); } + private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { + if (config.isClusteringEnabled()) { + Set fileGroupsInPendingClustering = + table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); + UpdateStrategy updateStrategy = (UpdateStrategy)ReflectionUtils + .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); + return (JavaRDD>)updateStrategy.handleUpdate(inputRecordsRDD); + } else { + return inputRecordsRDD; + } + } + @Override public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); @@ -101,14 +118,18 @@ public HoodieWriteMetadata> execute(JavaRDD WorkloadProfile profile = null; if (isWorkloadProfileNeeded()) { + context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile"); profile = new WorkloadProfile(buildProfile(inputRecordsRDD)); LOG.info("Workload profile :" + profile); saveWorkloadProfileMetadataToInflight(profile, instantTime); } + // handle records update with clustering + JavaRDD> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD); + // partition using the insert partitioner final Partitioner partitioner = getPartitioner(profile); - JavaRDD> partitionedRecords = partition(inputRecordsRDD, partitioner); + JavaRDD> partitionedRecords = partition(inputRecordsRDDWithClusteringUpdate, partitioner); JavaRDD writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> { if (WriteOperationType.isChangingRecords(operationType)) { return handleUpsertPartition(instantTime, partition, recordItr, partitioner); @@ -206,6 +227,7 @@ protected String getCommitActionType() { @Override protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect"); commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); } @@ -277,7 +299,7 @@ public Iterator> handleUpdate(String partitionPath, String fil return handleUpdateInternal(upsertHandle, fileId); } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) + protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId) throws IOException { if (upsertHandle.getOldFilePath() == null) { throw new HoodieUpsertException( @@ -287,11 +309,12 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle ups } // TODO(vc): This needs to be revisited - if (upsertHandle.getWriteStatus().getPartitionPath() == null) { + if (upsertHandle.getPartitionPath() == null) { LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", " - + upsertHandle.getWriteStatus()); + + upsertHandle.writeStatuses()); } - return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator(); + + return Collections.singletonList(upsertHandle.writeStatuses()).iterator(); } protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator> recordItr) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java index fb8b5f9cd3e05..2b00d47b01564 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java @@ -18,8 +18,6 @@ package org.apache.hudi.table.action.commit; -import java.util.Map; - import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; @@ -28,12 +26,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.BulkInsertPartitioner; - +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; +import java.util.Map; + public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { private final JavaRDD> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 9ccd66b2cd265..66fd68e15496c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -31,7 +31,6 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; - import org.apache.spark.api.java.JavaRDD; import java.util.List; @@ -59,25 +58,45 @@ public static SparkBulkInsertHelper newInstance() { } @Override - public HoodieWriteMetadata> bulkInsert(JavaRDD> inputRecords, - String instantTime, - HoodieTable>, JavaRDD, JavaRDD> table, - HoodieWriteConfig config, - BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, - boolean performDedupe, - Option> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsert(final JavaRDD> inputRecords, + final String instantTime, + final HoodieTable>, JavaRDD, JavaRDD> table, + final HoodieWriteConfig config, + final BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, + final boolean performDedupe, + final Option> userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); + //transition bulk_insert state to inflight + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, + table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), + config.shouldAllowMultiWriteOnSameInstant()); + // write new files + JavaRDD writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism()); + //update index + ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); + return result; + } + + @Override + public JavaRDD bulkInsert(JavaRDD> inputRecords, + String instantTime, + HoodieTable>, JavaRDD, JavaRDD> table, + HoodieWriteConfig config, + boolean performDedupe, + Option> userDefinedBulkInsertPartitioner, + boolean useWriterSchema, + int parallelism) { + // De-dupe/merge if needed JavaRDD> dedupedRecords = inputRecords; if (performDedupe) { dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, - config.getBulkInsertShuffleParallelism(), table); + parallelism, table); } final JavaRDD> repartitionedRecords; - final int parallelism = config.getBulkInsertShuffleParallelism(); BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); @@ -87,16 +106,11 @@ public HoodieWriteMetadata> bulkInsert(JavaRDD fileIDPrefixes = IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, - table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), - config.shouldAllowMultiWriteOnSameInstant()); - JavaRDD writeStatusRDD = repartitionedRecords .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, - partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true) + partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, useWriterSchema), true) .flatMap(List::iterator); - ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatusRDD, result); - return result; + return writeStatusRDD; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java index 01f9964b61bb2..83ead05085015 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java new file mode 100644 index 0000000000000..90bcdc9b9c141 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -0,0 +1,68 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SparkDeletePartitionCommitActionExecutor> + extends SparkInsertOverwriteCommitActionExecutor { + + private List partitions; + public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List partitions) { + super(context, config, table, instantTime,null, WriteOperationType.DELETE_PARTITION); + this.partitions = partitions; + } + + @Override + public HoodieWriteMetadata> execute() { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + HoodieTimer timer = new HoodieTimer().startTimer(); + Map> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct() + .mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + HoodieWriteMetadata result = new HoodieWriteMetadata(); + result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); + + result.setWriteStatuses(jsc.emptyRDD()); + this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + this.commitOnAutoCommit(result); + return result; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 2771a22c9f12c..7419a87f59e01 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -19,7 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -44,7 +44,14 @@ public class SparkInsertOverwriteCommitActionExecutor> inputRecordsRDD) { - super(context, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE); + this(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE); + } + + public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD, + WriteOperationType writeOperationType) { + super(context, config, table, instantTime, writeOperationType); this.inputRecordsRDD = inputRecordsRDD; } @@ -70,7 +77,7 @@ protected Map> getPartitionToReplacedFileIds(JavaRDD(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); } - private List getAllExistingFileIds(String partitionPath) { + protected List getAllExistingFileIds(String partitionPath) { // because new commit is not complete. it is safe to mark all existing file Ids as old files return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java index 6f8be79f94e2c..75dfbda30b7fb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java new file mode 100644 index 0000000000000..150d2f41665ee --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -0,0 +1,60 @@ +/* + * 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 org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SparkInsertOverwriteTableCommitActionExecutor> + extends SparkInsertOverwriteCommitActionExecutor { + + public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE); + } + + @Override + protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { + Map> partitionToExistingFileIds = new HashMap<>(); + List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + if (partitionPaths != null && partitionPaths.size() > 0) { + context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); + JavaRDD partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size()); + partitionToExistingFileIds = partitionPathRdd.mapToPair( + partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + } + return partitionToExistingFileIds; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java index 5f1a1ef5576dc..a197c91da946b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java @@ -59,10 +59,9 @@ public JavaRDD> deduplicateRecords(JavaRDD> reco }).reduceByKey((rec1, rec2) -> { @SuppressWarnings("unchecked") T reducedData = (T) rec1.getData().preCombine(rec2.getData()); - // we cannot allow the user to change the key or partitionPath, since that will affect - // everything - // so pick it from one of the records. - return new HoodieRecord(rec1.getKey(), reducedData); + HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); + + return new HoodieRecord(reducedKey, reducedData); }, parallelism).map(Tuple2::_2); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index b28c89a536469..000cfc7071c0d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.action.commit; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -129,6 +129,34 @@ private int addUpdateBucket(String partitionPath, String fileIdHint) { return bucket; } + /** + * Get the in pending clustering fileId for each partition path. + * @return partition path to pending clustering file groups id + */ + private Map> getPartitionPathToPendingClusteringFileGroupsId() { + Map> partitionPathToInPendingClusteringFileId = + table.getFileSystemView().getFileGroupsInPendingClustering() + .map(fileGroupIdAndInstantPair -> + Pair.of(fileGroupIdAndInstantPair.getKey().getPartitionPath(), fileGroupIdAndInstantPair.getKey().getFileId())) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toSet()))); + return partitionPathToInPendingClusteringFileId; + } + + /** + * Exclude small file handling for clustering since update path is not supported. + * @param pendingClusteringFileGroupsId pending clustering file groups id of partition + * @param smallFiles small files of partition + * @return smallFiles not in clustering + */ + private List filterSmallFilesInClustering(final Set pendingClusteringFileGroupsId, final List smallFiles) { + if (this.config.isClusteringEnabled()) { + return smallFiles.stream() + .filter(smallFile -> !pendingClusteringFileGroupsId.contains(smallFile.location.getFileId())).collect(Collectors.toList()); + } else { + return smallFiles; + } + } + private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) { // for new inserts, compute buckets depending on how many records we have for each partition Set partitionPaths = profile.getPartitionPaths(); @@ -140,11 +168,16 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) Map> partitionSmallFilesMap = getSmallFilesForPartitions(new ArrayList(partitionPaths), context); + Map> partitionPathToPendingClusteringFileGroupsId = getPartitionPathToPendingClusteringFileGroupsId(); + for (String partitionPath : partitionPaths) { WorkloadStat pStat = profile.getWorkloadStat(partitionPath); if (pStat.getNumInserts() > 0) { - List smallFiles = partitionSmallFilesMap.get(partitionPath); + List smallFiles = + filterSmallFilesInClustering(partitionPathToPendingClusteringFileGroupsId.getOrDefault(partitionPath, Collections.emptySet()), + partitionSmallFilesMap.get(partitionPath)); + this.smallFiles.addAll(smallFiles); LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles); @@ -157,7 +190,7 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) for (SmallFile smallFile : smallFiles) { long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize, totalUnassignedInserts); - if (recordsToAppend > 0 && totalUnassignedInserts > 0) { + if (recordsToAppend > 0) { // create a new bucket or re-use an existing bucket int bucket; if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) { @@ -185,7 +218,11 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket); for (int b = 0; b < insertBuckets; b++) { bucketNumbers.add(totalBuckets); - recordsPerBucket.add(totalUnassignedInserts / insertBuckets); + if (b < insertBuckets - 1) { + recordsPerBucket.add(insertRecordsPerBucket); + } else { + recordsPerBucket.add(totalUnassignedInserts - (insertBuckets - 1) * insertRecordsPerBucket); + } BucketInfo bucketInfo = new BucketInfo(); bucketInfo.bucketType = BucketType.INSERT; bucketInfo.partitionPath = partitionPath; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 505eabb88baef..17c19cebee991 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -18,13 +18,16 @@ package org.apache.hudi.table.action.compact; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.SparkTaskContextSupplier; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.HoodieBaseFile; @@ -49,10 +52,6 @@ import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -129,10 +128,18 @@ private List compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteT List logFiles = operation.getDeltaFileNames().stream().map( p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString()) .collect(toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), logFiles, - readerSchema, maxInstantTime, maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(), - config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), - config.getSpillableMapBasePath()); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(metaClient.getBasePath()) + .withLogFilePaths(logFiles) + .withReaderSchema(readerSchema) + .withLatestInstantTime(maxInstantTime) + .withMaxMemorySizeInBytes(maxMemoryPerCompaction) + .withReadBlocksLazily(config.getCompactionLazyBlockReadEnabled()) + .withReverseReader(config.getCompactionReverseLogReadEnabled()) + .withBufferSize(config.getMaxDFSStreamBufferSize()) + .withSpillableMapBasePath(config.getSpillableMapBasePath()) + .build(); if (!scanner.iterator().hasNext()) { return new ArrayList<>(); } @@ -171,7 +178,8 @@ private List compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteT @Override public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, HoodieTable>, JavaRDD, JavaRDD> hoodieTable, - HoodieWriteConfig config, String compactionCommitTime, Set fgIdsInPendingCompactions) + HoodieWriteConfig config, String compactionCommitTime, + Set fgIdsInPendingCompactionAndClustering) throws IOException { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); totalLogFiles = new LongAccumulator(); @@ -187,8 +195,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime); - List partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), - config.shouldAssumeDatePartitioning()); + List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath()); // filter the partition paths if needed to reduce list status partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths); @@ -205,7 +212,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, List operations = context.flatMap(partitionPaths, partitionPath -> { return fileSystemView .getLatestFileSlices(partitionPath) - .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())) + .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())) .map(s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); @@ -216,7 +223,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, // into meta files. Option dataFile = s.getBaseFile(); return new CompactionOperation(dataFile, partitionPath, logFiles, - config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); + config.getCompactionStrategy().captureMetrics(config, s)); }) .filter(c -> !c.getDeltaFileNames().isEmpty()); }, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); @@ -231,9 +238,9 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); ValidationUtils.checkArgument( compactionPlan.getOperations().stream().noneMatch( - op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), + op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " - + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions + + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering + ", Selected workload :" + compactionPlan); if (compactionPlan.getOperations().isEmpty()) { LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java index ebc3de5b804f2..5851b08c69e52 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java @@ -76,6 +76,7 @@ public HoodieWriteMetadata> execute() { JavaRDD statuses = compactor.compact(context, compactionPlan, table, config, instantTime); statuses.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata"); List updateStatusMap = statuses.map(WriteStatus::getStat).collect(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); for (HoodieWriteStat stat : updateStatusMap) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java index c5f6c1692c91e..34db0a7808b7d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java @@ -20,23 +20,25 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; 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.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.table.HoodieTable; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; @SuppressWarnings("checkstyle:LineLength") @@ -75,10 +77,13 @@ protected HoodieCompactionPlan scheduleCompaction() { LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor(); try { - return compactor.generateCompactionPlan(context, table, config, instantTime, - ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet())); + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + // exclude files in pending clustering from compaction. + fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java index 64d4c9ce85779..c92cd928474cc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java @@ -79,11 +79,10 @@ public Iterator> handleUpdate(String partitionPath, String fil LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId); return super.handleUpdate(partitionPath, fileId, recordItr); } else { - HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, partitionPath, fileId, recordItr, taskContextSupplier); appendHandle.doAppend(); - appendHandle.close(); - return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())).iterator(); + return Collections.singletonList(appendHandle.close()).iterator(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java index 9cf2434bc22a3..fcb3882b7bf5e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java @@ -18,10 +18,11 @@ package org.apache.hudi.table.action.rollback; -import org.apache.hudi.client.common.HoodieEngineContext; -import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -49,6 +50,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import scala.Tuple2; @@ -116,19 +118,29 @@ JavaPairRDD maybeDeleteAndCollectStats(HoodieEngineC .withDeletedFileResults(filesToDeletedStatus).build()); } case APPEND_ROLLBACK_BLOCK: { + String fileId = rollbackRequest.getFileId().get(); + String latestBaseInstant = rollbackRequest.getLatestBaseInstant().get(); + + // collect all log files that is supposed to be deleted with this rollback + Map writtenLogFileSizeMap = FSUtils.getAllLogFiles(metaClient.getFs(), + FSUtils.getPartitionPath(config.getBasePath(), rollbackRequest.getPartitionPath()), + fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), latestBaseInstant) + .collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen())); + Writer writer = null; try { writer = HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath())) - .withFileId(rollbackRequest.getFileId().get()) - .overBaseCommit(rollbackRequest.getLatestBaseInstant().get()).withFs(metaClient.getFs()) + .withFileId(fileId) + .overBaseCommit(latestBaseInstant) + .withFs(metaClient.getFs()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); // generate metadata if (doDelete) { Map header = generateHeader(instantToRollback.getTimestamp()); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); + writer.appendBlock(new HoodieCommandBlock(header)); } } catch (IOException | InterruptedException io) { throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io); @@ -149,9 +161,11 @@ JavaPairRDD maybeDeleteAndCollectStats(HoodieEngineC metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()), 1L ); + return new Tuple2<>(rollbackRequest.getPartitionPath(), HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath()) - .withRollbackBlockAppendResults(filesToNumBlocksRollback).build()); + .withRollbackBlockAppendResults(filesToNumBlocksRollback) + .withWrittenLogFileSizeMap(writtenLogFileSizeMap).build()); } default: throw new IllegalStateException("Unknown Rollback action " + rollbackRequest); @@ -159,7 +173,6 @@ JavaPairRDD maybeDeleteAndCollectStats(HoodieEngineC }); } - /** * Common method used for cleaning out base files under a partition path during rollback of a set of commits. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java index 965d8055943a5..611ec217a7759 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java @@ -66,8 +66,8 @@ protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() { @Override protected List executeRollbackUsingFileListing(HoodieInstant instantToRollback) { - List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - config.shouldAssumeDatePartitioning()); + List rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, + table.getMetaClient().getBasePath(), config); return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java index 065b22d787511..0f36cb85aa40d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java @@ -19,10 +19,13 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.IOType; @@ -32,10 +35,14 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; +import org.apache.hadoop.fs.FileStatus; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; import scala.Tuple2; @@ -52,6 +59,7 @@ public List execute(HoodieInstant instantToRollback) { MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp()); List markerFilePaths = markerFiles.allMarkerFilePaths(); int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1); + jsc.setJobGroup(this.getClass().getSimpleName(), "Rolling back using marker files"); return jsc.parallelize(markerFilePaths, parallelism) .map(markerFilePath -> { String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1); @@ -74,4 +82,11 @@ public List execute(HoodieInstant instantToRollback) { throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e); } } + + protected Map getWrittenLogFileSizeMap(String partitionPathStr, String baseCommitTime, String fileId) throws IOException { + // collect all log files that is supposed to be deleted with this rollback + return FSUtils.getAllLogFiles(table.getMetaClient().getFs(), + FSUtils.getPartitionPath(config.getBasePath(), partitionPathStr), fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime) + .collect(Collectors.toMap(HoodieLogFile::getFileStatus, value -> value.getFileStatus().getLen())); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java index 459ab128f0360..9486362558147 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java @@ -19,9 +19,9 @@ package org.apache.hudi.table.action.rollback; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java index 52849cb06f3d6..c30dd213e2f97 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java index 9c13c5a780ce8..accdee221ff73 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java @@ -18,7 +18,7 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.config.HoodieWriteConfig; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java index 7e3faf32b6528..a451956a2dcd7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.upgrade; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.IOType; @@ -92,8 +92,7 @@ private static void recreateMarkerFiles(final String commitInstantTime, // generate rollback stats List rollbackRequests; if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) { - rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(), - table.getConfig().shouldAssumeDatePartitioning()); + rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(context, table.getMetaClient().getBasePath(), table.getConfig()); } else { rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context); } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala index d1a4249f7962a..88101265de297 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala @@ -21,41 +21,15 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.common.model.HoodieKey import org.apache.spark.rdd.RDD import org.apache.spark.sql.avro.SchemaConverters -import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.{Dataset, Row, SparkSession} import scala.collection.JavaConverters._ object AvroConversionUtils { - def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = { - val avroSchema = convertStructTypeToAvroSchema(df.schema, structName, recordNamespace) - createRdd(df, avroSchema, structName, recordNamespace) - } - - def createRdd(df: DataFrame, avroSchema: Schema, structName: String, recordNamespace: String) - : RDD[GenericRecord] = { - // Use the Avro schema to derive the StructType which has the correct nullability information - val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType] - val encoder = RowEncoder.apply(dataType).resolveAndBind() - df.queryExecution.toRdd.map(encoder.fromRow) - .mapPartitions { records => - if (records.isEmpty) Iterator.empty - else { - val convertor = AvroConversionHelper.createConverterToAvro(dataType, structName, recordNamespace) - records.map { x => convertor(x).asInstanceOf[GenericRecord] } - } - } - } - - def createRddForDeletes(df: DataFrame, rowField: String, partitionField: String): RDD[HoodieKey] = { - df.rdd.map(row => new HoodieKey(row.getAs[String](rowField), row.getAs[String](partitionField))) - } - def createDataFrame(rdd: RDD[GenericRecord], schemaStr: String, ss: SparkSession): Dataset[Row] = { if (rdd.isEmpty()) { ss.emptyDataFrame diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java index d04a2df1b4eb4..0599ee8d82b79 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java @@ -99,8 +99,9 @@ public void testSavepointAndRollback() throws Exception { statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); + HoodieWriteConfig config = getConfig(); List partitionPaths = - FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning()); + FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), cfg.getBasePath()); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); final BaseFileOnlyView view1 = table.getBaseFileOnlyView(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java index 03328ddddc142..e59a95009f212 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java @@ -37,6 +37,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -70,6 +71,11 @@ public void setUp() throws Exception { client = new CompactionAdminClient(context, basePath); } + @AfterEach + public void cleanUp() throws Exception { + cleanupResources(); + } + @Test public void testUnscheduleCompactionPlan() throws Exception { int numEntriesPerInstant = 10; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index bbb40488bb04e..48a4d12c08df0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -21,43 +21,56 @@ import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; +import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkWriteHelper; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -73,6 +86,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -80,6 +94,8 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.Properties; import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; import static org.apache.hudi.common.testutils.FileCreateUtils.getBaseFileCountsForPaths; @@ -91,6 +107,8 @@ import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys; import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet; import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet; +import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY; +import static org.apache.hudi.config.HoodieClusteringConfig.DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -104,11 +122,17 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { private static final Logger LOG = LogManager.getLogger(TestHoodieClientOnCopyOnWriteStorage.class); + private static final Map STRATEGY_PARAMS = new HashMap() { + { + put("sortColumn", "record_key"); + } + }; + private HoodieTestTable testTable; @BeforeEach public void setUpTestTable() { - testTable = HoodieWriteableTestTable.of(metaClient); + testTable = HoodieSparkWriteableTestTable.of(metaClient); } /** @@ -241,6 +265,7 @@ private void testDeduplication( when(index.isGlobal()).thenReturn(true); List> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); assertEquals(1, dedupedRecs.size()); + assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath()); assertNodupesWithinPartition(dedupedRecs); // non-Global dedup should be done based on both recordKey and partitionPath @@ -376,6 +401,53 @@ private void testUpsertsInternal(HoodieWriteConfig config, instants.get(3)); assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "006"), instants.get(4)); + + final HoodieWriteConfig cfg = hoodieWriteConfig; + final String instantTime = "007"; + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); + String basePathStr = basePath; + HoodieTable table = getHoodieTable(metaClient, cfg); + jsc.parallelize(Arrays.asList(1)).map(e -> { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails( + metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get()).get(), + HoodieCommitMetadata.class); + String filePath = commitMetadata.getPartitionToWriteStats().values().stream() + .flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny() + .map(ee -> ee.getPath()).orElse(null); + String partitionPath = commitMetadata.getPartitionToWriteStats().values().stream() + .flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny() + .map(ee -> ee.getPartitionPath()).orElse(null); + Path parquetFilePath = new Path(basePathStr, filePath); + HoodieBaseFile baseFile = new HoodieBaseFile(parquetFilePath.toString()); + + try { + HoodieMergeHandle handle = new HoodieMergeHandle(cfg, instantTime, table, new HashMap<>(), + partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier()); + WriteStatus writeStatus = new WriteStatus(false, 0.0); + writeStatus.setStat(new HoodieWriteStat()); + writeStatus.getStat().setNumWrites(0); + handle.performMergeDataValidationCheck(writeStatus); + } catch (HoodieCorruptedDataException e1) { + fail("Exception not expected because merge validation check is disabled"); + } + + try { + final String newInstantTime = "006"; + cfg.getProps().setProperty("hoodie.merge.data.validation.enabled", "true"); + HoodieWriteConfig cfg2 = HoodieWriteConfig.newBuilder().withProps(cfg.getProps()).build(); + HoodieMergeHandle handle = new HoodieMergeHandle(cfg2, newInstantTime, table, new HashMap<>(), + partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier()); + WriteStatus writeStatus = new WriteStatus(false, 0.0); + writeStatus.setStat(new HoodieWriteStat()); + writeStatus.getStat().setNumWrites(0); + handle.performMergeDataValidationCheck(writeStatus); + fail("The above line should have thrown an exception"); + } catch (HoodieCorruptedDataException e2) { + // expected + } + return true; + }).collect(); } /** @@ -627,6 +699,70 @@ private void assertActualAndExpectedPartitionPathRecordKeyMatches(Set, List> insertBatchRecords(SparkRDDWriteClient client, String commitTime, + Integer recordNum, int expectStatueSize) { + client.startCommitWithTime(commitTime); + List inserts1 = dataGen.generateInserts(commitTime, recordNum); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); + List statuses = client.upsert(insertRecordsRDD1, commitTime).collect(); + assertNoWriteErrors(statuses); + assertEquals(expectStatueSize, statuses.size(), "check expect statue size."); + return Pair.of(statuses, inserts1); + } + + @Test + public void testUpdateRejectForClustering() throws IOException { + final String testPartitionPath = "2016/09/26"; + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + Properties props = new Properties(); + props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, "true"); + HoodieWriteConfig config = getSmallInsertWriteConfig(100, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); + + //1. insert to generate 2 file group + String commitTime1 = "001"; + Pair, List> upsertResult = insertBatchRecords(client, commitTime1, 600, 2); + List inserts1 = upsertResult.getValue(); + List fileGroupIds1 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(2, fileGroupIds1.size()); + + // 2. generate clustering plan for fileGroupIds1 file groups + String commitTime2 = "002"; + List> firstInsertFileSlicesList = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getAllFileSlices().collect(Collectors.toList())).collect(Collectors.toList()); + List[] fileSlices = (List[])firstInsertFileSlicesList.toArray(new List[firstInsertFileSlicesList.size()]); + createRequestedReplaceInstant(this.metaClient, commitTime2, fileSlices); + + // 3. insert one record with no updating reject exception, and not merge the small file, just generate a new file group + String commitTime3 = "003"; + insertBatchRecords(client, commitTime3, 1, 1).getKey(); + List fileGroupIds2 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(3, fileGroupIds2.size()); + + // 4. update one record for the clustering two file groups, throw reject update exception + String commitTime4 = "004"; + client.startCommitWithTime(commitTime4); + List insertsAndUpdates3 = new ArrayList<>(); + insertsAndUpdates3.addAll(dataGen.generateUpdates(commitTime4, inserts1)); + String assertMsg = String.format("Not allowed to update the clustering files in partition: %s " + + "For pending clustering operations, we are not going to support update for now.", testPartitionPath); + assertThrows(HoodieUpsertException.class, () -> { + writeClient.upsert(jsc.parallelize(insertsAndUpdates3, 1), commitTime3).collect(); }, assertMsg); + + // 5. insert one record with no updating reject exception, will merge the small file + String commitTime5 = "005"; + List statuses = insertBatchRecords(client, commitTime5, 1, 1).getKey(); + fileGroupIds2.removeAll(fileGroupIds1); + assertEquals(fileGroupIds2.get(0), statuses.get(0).getFileId()); + List firstInsertFileGroupIds4 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(3, firstInsertFileGroupIds4.size()); + } + /** * Test scenario of new file-group getting added during upsert(). */ @@ -888,6 +1024,65 @@ public void testDeletesWithDeleteApi() throws Exception { testDeletes(client, updateBatch3.getRight(), 10, file1, "007", 140, keysSoFar); } + @Test + public void testSimpleClustering() throws Exception { + // setup clustering config + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + testClustering(clusteringConfig); + } + + @Test + public void testClusteringWithSortColumns() throws Exception { + // setup clustering config + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringSortColumns("_hoodie_record_key") + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + testClustering(clusteringConfig); + } + + private void testClustering(HoodieClusteringConfig clusteringConfig) throws Exception { + // create config to not update small files. + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false, 10); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + dataGen = new HoodieTestDataGenerator(); + String commitTime = "100"; + List records1 = dataGen.generateInserts(commitTime, 200); + List statuses1 = writeAndVerifyBatch(client, records1, commitTime); + Set fileIds1 = getFileGroupIdsFromWriteStatus(statuses1); + + commitTime = "200"; + List records2 = dataGen.generateInserts(commitTime, 200); + List statuses2 = writeAndVerifyBatch(client, records2, commitTime); + Set fileIds2 = getFileGroupIdsFromWriteStatus(statuses2); + //verify new files are created for 2nd write + Set fileIdIntersection = new HashSet<>(fileIds1); + fileIdIntersection.retainAll(fileIds2); + assertEquals(0, fileIdIntersection.size()); + + config = getConfigBuilder().withClusteringConfig(clusteringConfig).build(); + + // create client with new config. + client = getHoodieWriteClient(config, false); + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + HoodieWriteMetadata> clusterMetadata = client.cluster(clusteringCommitTime, true); + List allRecords = Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList()); + verifyRecordsWritten(clusteringCommitTime, allRecords, clusterMetadata.getWriteStatuses().collect()); + Set insertedFileIds = new HashSet<>(); + insertedFileIds.addAll(fileIds1); + insertedFileIds.addAll(fileIds2); + + Set replacedFileIds = new HashSet<>(); + clusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles -> + partitionFiles.getValue().stream().forEach(file -> + replacedFileIds.add(new HoodieFileGroupId(partitionFiles.getKey(), file)))); + assertEquals(insertedFileIds, replacedFileIds); + } + + private Set getFileGroupIdsFromWriteStatus(List statuses) { + return statuses.stream().map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).collect(Collectors.toSet()); + } + /** * Test scenario of writing more file groups than existing number of file groups in partition. */ @@ -908,7 +1103,7 @@ public void testInsertOverwritePartitionHandlingWithFewerRecords() throws Except * Test scenario of writing similar number file groups in partition. */ @Test - public void testInsertOverwritePartitionHandlinWithSimilarNumberOfRecords() throws Exception { + public void testInsertOverwritePartitionHandlingWithSimilarNumberOfRecords() throws Exception { verifyInsertOverwritePartitionHandling(3000, 3000); } @@ -925,14 +1120,9 @@ private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); // Do Inserts - String commitTime1 = "001"; - client.startCommitWithTime(commitTime1); - List inserts1 = dataGen.generateInserts(commitTime1, batch1RecordsCount); - JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); - List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); - assertNoWriteErrors(statuses); - Set batch1Buckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); - verifyRecordsWritten(commitTime1, inserts1, statuses); + String commit1 = "001"; + List statuses = writeAndVerifyBatch(client, dataGen.generateInserts(commit1, batch1RecordsCount), commit1); + Set batch1Buckets = getFileIdsFromWriteStatus(statuses); // Do Insert Overwrite String commitTime2 = "002"; @@ -949,6 +1139,113 @@ private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int verifyRecordsWritten(commitTime2, inserts2, statuses); } + private Set getFileIdsFromWriteStatus(List statuses) { + return statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); + } + + /** + * Test scenario of writing fewer file groups for first partition than second an third partition. + */ + @Test + public void verifyDeletePartitionsHandlingWithFewerRecordsFirstPartition() throws Exception { + verifyDeletePartitionsHandling(1000, 3000, 3000); + } + + /** + * Test scenario of writing similar number file groups in partition. + */ + @Test + public void verifyDeletePartitionsHandlingWithSimilarNumberOfRecords() throws Exception { + verifyDeletePartitionsHandling(3000, 3000, 3000); + } + + /** + * Test scenario of writing more file groups for first partition than second an third partition. + */ + @Test + public void verifyDeletePartitionsHandlingHandlingWithFewerRecordsSecondThirdPartition() throws Exception { + verifyDeletePartitionsHandling(3000, 1000, 1000); + } + + private Set insertPartitionRecordsWithCommit(SparkRDDWriteClient client, int recordsCount, String commitTime1, String partitionPath) { + client.startCommitWithTime(commitTime1); + List inserts1 = dataGen.generateInsertsForPartition(commitTime1, recordsCount, partitionPath); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); + List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); + assertNoWriteErrors(statuses); + Set batchBuckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); + verifyRecordsWritten(commitTime1, inserts1, statuses); + return batchBuckets; + } + + private Set deletePartitionWithCommit(SparkRDDWriteClient client, String commitTime, List deletePartitionPath) { + client.startCommitWithTime(commitTime, HoodieTimeline.REPLACE_COMMIT_ACTION); + HoodieWriteResult writeResult = client.deletePartitions(deletePartitionPath, commitTime); + Set deletePartitionReplaceFileIds = + writeResult.getPartitionToReplaceFileIds().entrySet() + .stream().flatMap(entry -> entry.getValue().stream()).collect(Collectors.toSet()); + return deletePartitionReplaceFileIds; + } + + /** + * 1) Do write1 (upsert) with 'batch1RecordsCount' number of records for first partition. + * 2) Do write2 (upsert) with 'batch2RecordsCount' number of records for second partition. + * 3) Do write3 (upsert) with 'batch3RecordsCount' number of records for third partition. + * 4) delete first partition and check result. + * 5) delete second and third partition and check result. + * + */ + private void verifyDeletePartitionsHandling(int batch1RecordsCount, int batch2RecordsCount, int batch3RecordsCount) throws Exception { + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + dataGen = new HoodieTestDataGenerator(); + + // Do Inserts for DEFAULT_FIRST_PARTITION_PATH + String commitTime1 = "001"; + Set batch1Buckets = + this.insertPartitionRecordsWithCommit(client, batch1RecordsCount, commitTime1, DEFAULT_FIRST_PARTITION_PATH); + + // Do Inserts for DEFAULT_SECOND_PARTITION_PATH + String commitTime2 = "002"; + Set batch2Buckets = + this.insertPartitionRecordsWithCommit(client, batch2RecordsCount, commitTime2, DEFAULT_SECOND_PARTITION_PATH); + + // Do Inserts for DEFAULT_THIRD_PARTITION_PATH + String commitTime3 = "003"; + Set batch3Buckets = + this.insertPartitionRecordsWithCommit(client, batch3RecordsCount, commitTime3, DEFAULT_THIRD_PARTITION_PATH); + + // delete DEFAULT_FIRST_PARTITION_PATH + String commitTime4 = "004"; + Set deletePartitionReplaceFileIds1 = + deletePartitionWithCommit(client, commitTime4, Arrays.asList(DEFAULT_FIRST_PARTITION_PATH)); + assertEquals(batch1Buckets, deletePartitionReplaceFileIds1); + List baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_FIRST_PARTITION_PATH)); + assertEquals(0, baseFiles.size()); + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_SECOND_PARTITION_PATH)); + assertTrue(baseFiles.size() > 0); + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_THIRD_PARTITION_PATH)); + assertTrue(baseFiles.size() > 0); + + // delete DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH + String commitTime5 = "005"; + Set deletePartitionReplaceFileIds2 = + deletePartitionWithCommit(client, commitTime5, Arrays.asList(DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH)); + Set expectedFileId = new HashSet<>(); + expectedFileId.addAll(batch2Buckets); + expectedFileId.addAll(batch3Buckets); + assertEquals(expectedFileId, deletePartitionReplaceFileIds2); + + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_FIRST_PARTITION_PATH), + String.format("%s/%s/*", basePath, DEFAULT_SECOND_PARTITION_PATH), + String.format("%s/%s/*", basePath, DEFAULT_THIRD_PARTITION_PATH)); + assertEquals(0, baseFiles.size()); + } + /** * Verify data in parquet files matches expected records and commit time. */ @@ -969,6 +1266,15 @@ private void verifyRecordsWritten(String commitTime, List expected } } + private List writeAndVerifyBatch(SparkRDDWriteClient client, List inserts, String commitTime) { + client.startCommitWithTime(commitTime); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts, 2); + List statuses = client.upsert(insertRecordsRDD1, commitTime).collect(); + assertNoWriteErrors(statuses); + verifyRecordsWritten(commitTime, inserts, statuses); + return statuses; + } + private Pair, List> testUpdates(String instantTime, SparkRDDWriteClient client, int sizeToInsertAndUpdate, int expectedTotalRecords) throws IOException { @@ -1336,16 +1642,45 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { * Build Hoodie Write Config for small data file sizes. */ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema) { - HoodieWriteConfig.Builder builder = getConfigBuilder(useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA); + return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, dataGen.getEstimatedFileSizeInBytes(150)); + } + + /** + * Build Hoodie Write Config for specified small file sizes. + */ + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema, long smallFileSize) { + String schemaStr = useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA; + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize); + } + + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize) { + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, new Properties()); + } + + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, Properties props) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr); return builder .withCompactionConfig( HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(dataGen.getEstimatedFileSizeInBytes(150)) + .compactionSmallFileSize(smallFileSize) .insertSplitSize(insertSplitSize).build()) .withStorageConfig( HoodieStorageConfig.newBuilder() .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) + .withProps(props) .build(); } + + protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient metaClient, String clusterTime, List[] fileSlices) throws IOException { + HoodieClusteringPlan clusteringPlan = + ClusteringUtils.createClusteringPlan(DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS, STRATEGY_PARAMS, fileSlices, Collections.emptyMap()); + + HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime); + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); + metaClient.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + return clusteringInstant; + } + } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 8ee0c163409a5..ecf38d8a9017c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -107,10 +107,33 @@ public void testSchemaCompatibilityBasic() throws Exception { assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema), "Swapped fields are not compatible"); - String typeChangeSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + String typeChangeSchemaDisallowed = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA.replace("string", "boolean") + TRIP_SCHEMA_SUFFIX; - assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchema), - "Field type change is not compatible"); + assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchemaDisallowed), + "Incompatible field type change is not allowed"); + + // Array of allowed schema field type transitions + String[][] allowedFieldChanges = { + {"string", "bytes"}, {"bytes", "string"}, + {"int", "long"}, {"int", "float"}, {"long", "float"}, + {"int", "double"}, {"float", "double"}, {"long", "double"}}; + for (String[] fieldChange : allowedFieldChanges) { + String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[0]) + TRIP_SCHEMA_SUFFIX; + String toSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[1]) + TRIP_SCHEMA_SUFFIX; + assertTrue(TableSchemaResolver.isSchemaCompatible(fromSchema, toSchema), + "Compatible field type change is not allowed"); + if (!fieldChange[0].equals("byte") && fieldChange[1].equals("byte")) { + assertFalse(TableSchemaResolver.isSchemaCompatible(toSchema, fromSchema), + "Incompatible field type change is allowed"); + } + } + + // Names and aliases should match + String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; + String toSchema = TRIP_SCHEMA_PREFIX.replace("triprec", "new_triprec") + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; + assertFalse(TableSchemaResolver.isSchemaCompatible(fromSchema, toSchema), "Field names should match"); + assertFalse(TableSchemaResolver.isSchemaCompatible(toSchema, fromSchema), "Field names should match"); + assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED), "Added field with default is compatible (Evolved Schema)"); @@ -458,7 +481,7 @@ private List convertToSchema(List records, String sc GenericRecord payload; try { payload = (GenericRecord)r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); - GenericRecord newPayload = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(payload, newSchema); + GenericRecord newPayload = HoodieAvroUtils.rewriteRecord(payload, newSchema); return new HoodieRecord(key, new RawTripTestPayload(newPayload.toString(), key.getRecordKey(), key.getPartitionPath(), schemaStr)); } catch (IOException e) { throw new RuntimeException("Conversion to new schema failed"); @@ -474,6 +497,7 @@ private HoodieWriteConfig getWriteConfig(String schema) { .build(); } + @Override protected HoodieTableType getTableType() { return tableType; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index 9a8d7e0c88898..9b53de1d682ca 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -84,10 +84,10 @@ private WriteStatus prepareFirstRecordCommit(List recordsStrs) throws IO } Map insertRecordMap = insertRecords.stream() .collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity())); - HoodieCreateHandle createHandle = + HoodieCreateHandle createHandle = new HoodieCreateHandle(config, "100", table, insertRecords.get(0).getPartitionPath(), "f1-0", insertRecordMap, supplier); createHandle.write(); - return createHandle.close(); + return createHandle.close().get(0); }).collect(); final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100")); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java index 834229b683f1e..81effaa2a7999 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java @@ -18,17 +18,23 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.testutils.HoodieClientTestBase; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -40,6 +46,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase { + private static final Comparator> KEY_COMPARATOR = + Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())); public static JavaRDD generateTestRecordsForBulkInsert(JavaSparkContext jsc) { HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); @@ -69,9 +77,10 @@ private static Stream configParams() { return Stream.of(data).map(Arguments::of); } - private void verifyRecordAscendingOrder(List records) { - List expectedRecords = new ArrayList<>(records); - Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey()))); + private void verifyRecordAscendingOrder(List> records, + Option>> comparator) { + List> expectedRecords = new ArrayList<>(records); + Collections.sort(expectedRecords, comparator.orElse(KEY_COMPARATOR)); assertEquals(expectedRecords, records); } @@ -79,19 +88,28 @@ private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner JavaRDD records, boolean isGloballySorted, boolean isLocallySorted, Map expectedPartitionNumRecords) { + testBulkInsertInternalPartitioner(partitioner, records, isGloballySorted, isLocallySorted, expectedPartitionNumRecords, Option.empty()); + } + + private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner, + JavaRDD records, + boolean isGloballySorted, boolean isLocallySorted, + Map expectedPartitionNumRecords, + Option>> comparator) { int numPartitions = 2; - JavaRDD actualRecords = (JavaRDD) partitioner.repartitionRecords(records, numPartitions); + JavaRDD> actualRecords = + (JavaRDD>) partitioner.repartitionRecords(records, numPartitions); assertEquals(numPartitions, actualRecords.getNumPartitions()); - List collectedActualRecords = actualRecords.collect(); + List> collectedActualRecords = actualRecords.collect(); if (isGloballySorted) { // Verify global order - verifyRecordAscendingOrder(collectedActualRecords); + verifyRecordAscendingOrder(collectedActualRecords, comparator); } else if (isLocallySorted) { // Verify local order actualRecords.mapPartitions(partition -> { - List partitionRecords = new ArrayList<>(); + List> partitionRecords = new ArrayList<>(); partition.forEachRemaining(partitionRecords::add); - verifyRecordAscendingOrder(partitionRecords); + verifyRecordAscendingOrder(partitionRecords, comparator); return Collections.emptyList().iterator(); }).collect(); } @@ -118,4 +136,35 @@ public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode, testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode), records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2)); } + + @Test + public void testCustomColumnSortPartitioner() throws Exception { + String[] sortColumns = new String[] {"rider"}; + Comparator> columnComparator = getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns); + + JavaRDD records1 = generateTestRecordsForBulkInsert(jsc); + JavaRDD records2 = generateTripleTestRecordsForBulkInsert(jsc); + testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA), + records1, true, true, generateExpectedPartitionNumRecords(records1), Option.of(columnComparator)); + testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA), + records2, true, true, generateExpectedPartitionNumRecords(records2), Option.of(columnComparator)); + } + + private Comparator> getCustomColumnComparator(Schema schema, String[] sortColumns) { + Comparator> comparator = Comparator.comparing(record -> { + try { + GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + StringBuilder sb = new StringBuilder(); + for (String col : sortColumns) { + sb.append(genericRecord.get(col)); + } + + return sb.toString(); + } catch (IOException e) { + throw new HoodieIOException("unable to read value for " + sortColumns); + } + }); + + return comparator; + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java index f10e845f05b55..a18313b278656 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java @@ -39,7 +39,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.Assertions; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.avro.Schema; @@ -280,7 +280,7 @@ public void testTagLocationAndFetchRecordLocations(IndexType indexType) throws E } // We create three parquet file, each having one record. (two different partitions) - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); String fileId1 = testTable.addCommit("001").getFileIdWithInserts(p1, record1); String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2); String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4); @@ -337,7 +337,7 @@ public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath(IndexT writeClient = getHoodieWriteClient(config); index = writeClient.getIndex(); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); final String p1 = "2016/01/31"; final String p2 = "2016/02/28"; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 9175ebde51333..0af28cc8d6ad9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -20,7 +20,7 @@ package org.apache.hudi.index; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 8446698305846..b325eb6b1c404 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -33,7 +33,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.avro.Schema; import org.apache.hadoop.fs.Path; @@ -105,7 +105,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files // "2016/01/21": 0 file @@ -222,7 +222,7 @@ public void testCheckUUIDsAgainstOneFile() throws Exception { // record2, record3). BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record3.getRecordKey()); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter); String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2); String filename = testTable.getBaseFileNameById(fileId); @@ -298,7 +298,7 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); @@ -363,7 +363,7 @@ public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean // Also create the metadata and config HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Let's tag SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config); @@ -432,7 +432,7 @@ public void testBloomFilterFalseError(boolean rangePruning, boolean treeFilterin BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); filter.add(record2.getRecordKey()); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter); String fileId = testTable.addCommit("000").getFileIdWithInserts("2016/01/31", record1); assertTrue(filter.mightContain(record1.getRecordKey())); assertTrue(filter.mightContain(record2.getRecordKey())); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index c18eeb1d770c8..ff87f967a1d0d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -27,7 +27,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.avro.Schema; import org.apache.spark.api.java.JavaPairRDD; @@ -76,7 +76,7 @@ public void testLoadInvolvedFiles() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files, along with the meta file // "2016/01/21": 0 file @@ -180,7 +180,7 @@ public void testTagLocation() throws Exception { HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build(); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); // Create some partitions, and put some files, along with the meta file // "2016/01/21": 0 file @@ -261,7 +261,7 @@ public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception { .build(); SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA); final String p1 = "2016/01/31"; final String p2 = "2016/02/28"; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java index 2eb672a00bd4c..85b5743f4a4a6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java @@ -18,8 +18,10 @@ package org.apache.hudi.index.hbase; +import avro.shaded.com.google.common.collect.Maps; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -119,6 +121,8 @@ public static void init() throws Exception { public void setUp() throws Exception { hadoopConf = jsc().hadoopConfiguration(); hadoopConf.addResource(utility.getConfiguration()); + // reInit the context here to keep the hadoopConf the same with that in this class + context = new HoodieSparkEngineContext(jsc()); metaClient = getHoodieMetaClient(hadoopConf, basePath()); dataGen = new HoodieTestDataGenerator(); } @@ -307,6 +311,125 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count()); } + /* + * Test case to verify that for taglocation entries present in HBase, if the corresponding commit instant is missing + * in timeline and the commit is not archived, taglocation would reset the current record location to null. + */ + @Test + public void testSimpleTagLocationWithInvalidCommit() throws Exception { + // Load to memory + HoodieWriteConfig config = getConfig(); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + String newCommitTime = writeClient.startCommit(); + // make a commit with 199 records + JavaRDD writeRecords = generateAndCommitRecords(writeClient, 199, newCommitTime); + + // make a second commit with a single record + String invalidCommit = writeClient.startCommit(); + JavaRDD invalidWriteRecords = generateAndCommitRecords(writeClient, 1, invalidCommit); + + // verify location is tagged. + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD0 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + assert (javaRDD0.collect().size() == 1); // one record present + assert (javaRDD0.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 1); // it is tagged + assert (javaRDD0.collect().get(0).getCurrentLocation().getInstantTime().equals(invalidCommit)); + + // rollback the invalid commit, so that hbase will be left with a stale entry. + writeClient.rollback(invalidCommit); + + // Now tagLocation for the valid records, hbaseIndex should tag them + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords, context(), hoodieTable); + assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 199); + + // tagLocation for the invalid record - commit is not present in timeline due to rollback. + JavaRDD javaRDD2 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + assert (javaRDD2.collect().size() == 1); // one record present + assert (javaRDD2.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); // it is not tagged + } + + /* + * Test case to verify that taglocation() uses the commit timeline to validate the commitTS stored in hbase. + * When CheckIfValidCommit() in HbaseIndex uses the incorrect timeline filtering, this test would fail. + */ + @Test + public void testEnsureTagLocationUsesCommitTimeline() throws Exception { + // Load to memory + HoodieWriteConfig config = getConfig(); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + String commitTime1 = writeClient.startCommit(); + JavaRDD writeRecords1 = generateAndCommitRecords(writeClient, 20, commitTime1); + + // rollback the commit - leaves a clean file in timeline. + writeClient.rollback(commitTime1); + + // create a second commit with 20 records + metaClient = HoodieTableMetaClient.reload(metaClient); + generateAndCommitRecords(writeClient, 20); + + // Now tagLocation for the first set of rolledback records, hbaseIndex should tag them + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 20); + } + + private JavaRDD generateAndCommitRecords(SparkRDDWriteClient writeClient, int numRecs) throws Exception { + String commitTime = writeClient.startCommit(); + return generateAndCommitRecords(writeClient, numRecs, commitTime); + } + + private JavaRDD generateAndCommitRecords(SparkRDDWriteClient writeClient, + int numRecs, String commitTime) throws Exception { + // first batch of records + List records = dataGen.generateInserts(commitTime, numRecs); + JavaRDD writeRecords = jsc().parallelize(records, 1); + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Insert records + JavaRDD writeStatues = writeClient.upsert(writeRecords, commitTime); + assertNoWriteErrors(writeStatues.collect()); + + // commit this upsert + writeClient.commit(commitTime, writeStatues); + + return writeRecords; + } + + // Verify hbase is tagging records belonging to an archived commit as valid. + @Test + public void testHbaseTagLocationForArchivedCommits() throws Exception { + // Load to memory + Map params = Maps.newHashMap(); + params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1"); + params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "3"); + params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2"); + HoodieWriteConfig config = getConfigBuilder(100, false).withProps(params).build(); + + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + // make first commit with 20 records + JavaRDD writeRecords1 = generateAndCommitRecords(writeClient, 20); + + // Make 3 additional commits, so that first commit is archived + for (int nCommit = 0; nCommit < 3; nCommit++) { + generateAndCommitRecords(writeClient, 20); + } + + // tagLocation for the first set of records (for the archived commit), hbaseIndex should tag them as valid + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + assertEquals(20, javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size()); + } + @Test public void testTotalGetsBatching() throws Exception { HoodieWriteConfig config = getConfig(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java index 3a7d468e770d8..ff012b0499cbd 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java @@ -18,7 +18,7 @@ package org.apache.hudi.io; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -35,7 +35,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.junit.jupiter.api.AfterEach; @@ -87,7 +87,7 @@ public void testFetchHandle() throws Exception { List records = dataGen.generateInserts(makeNewCommitTime(), 100); Map> partitionRecordsMap = recordsToPartitionRecordsMap(records); HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS); + HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS); Map, List>> expectedList = writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable); @@ -95,15 +95,15 @@ public void testFetchHandle() throws Exception { for (Tuple2 entry : partitionPathFileIdPairs) { HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2)); - Iterator> result = fetcherHandle.locations(); + Iterator> result = fetcherHandle.locations().iterator(); List> actualList = new ArrayList<>(); - result.forEachRemaining(actualList::add); + result.forEachRemaining(x -> actualList.add(new Tuple2<>(x.getLeft(), x.getRight()))); assertEquals(expectedList.get(new Tuple2<>(entry._1, entry._2.getFileId())), actualList); } } private Map, List>> writeToParquetAndGetExpectedRecordLocations( - Map> partitionRecordsMap, HoodieWriteableTestTable testTable) throws Exception { + Map> partitionRecordsMap, HoodieSparkWriteableTestTable testTable) throws Exception { Map, List>> expectedList = new HashMap<>(); for (Map.Entry> entry : partitionRecordsMap.entrySet()) { int totalRecordsPerPartition = entry.getValue().size(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java index c7a313ab96b34..edce77772d407 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java @@ -72,7 +72,7 @@ public void tearDown() throws Exception { } @Test - public void testRowCreateHandle() throws IOException { + public void testRowCreateHandle() throws Exception { // init config and table HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); @@ -113,7 +113,7 @@ public void testRowCreateHandle() throws IOException { * should be thrown. */ @Test - public void testGlobalFailure() throws IOException { + public void testGlobalFailure() throws Exception { // init config and table HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); @@ -179,7 +179,8 @@ public void testInstantiationFailure() throws IOException { } } - private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset inputRows, HoodieRowCreateHandle handle) throws IOException { + private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset inputRows, HoodieRowCreateHandle handle) + throws Exception { List internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER); // issue writes for (InternalRow internalRow : internalRows) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java index 37b8cdca88bbc..2b344db7e1322 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java @@ -35,7 +35,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.List; import java.util.Random; import java.util.UUID; @@ -64,7 +63,7 @@ public void tearDown() throws Exception { } @Test - public void endToEndTest() throws IOException { + public void endToEndTest() throws Exception { HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build(); for (int i = 0; i < 5; i++) { // init write support and parquet config diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java new file mode 100644 index 0000000000000..c238fc04b84bb --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -0,0 +1,951 @@ +/* + * 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.metadata; + +import org.apache.hudi.client.HoodieWriteResult; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieMetricsConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieBackedMetadata extends HoodieClientTestHarness { + + private static final Logger LOG = LogManager.getLogger(TestHoodieBackedMetadata.class); + + @TempDir + public java.nio.file.Path tempFolder; + + private String metadataTableBasePath; + + private HoodieTableType tableType; + + public void init(HoodieTableType tableType) throws IOException { + this.tableType = tableType; + initPath(); + initSparkContexts("TestHoodieMetadata"); + initFileSystem(); + fs.mkdirs(new Path(basePath)); + initMetaClient(tableType); + initTestDataGenerator(); + metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + + } + + @AfterEach + public void clean() throws IOException { + cleanupResources(); + } + + /** + * Metadata Table should not be created unless it is enabled in config. + */ + @Test + public void testDefaultNoMetadataTable() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Metadata table should not exist until created for the first time + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not exist"); + assertThrows(TableNotFoundException.class, () -> new HoodieTableMetaClient(hadoopConf, metadataTableBasePath)); + + // Metadata table is not created if disabled by config + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + client.startCommitWithTime("001"); + assertFalse(fs.exists(new Path(metadataTableBasePath)), "Metadata table should not be created"); + assertThrows(TableNotFoundException.class, () -> new HoodieTableMetaClient(hadoopConf, metadataTableBasePath)); + } + + // Metadata table created when enabled by config & sync is called + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + client.startCommitWithTime("001"); + client.syncTableMetadata(); + assertTrue(fs.exists(new Path(metadataTableBasePath))); + validateMetadata(client); + } + } + + /** + * Only valid partition directories are added to the metadata. + */ + @Test + public void testOnlyValidPartitionsAdded() throws Exception { + // This test requires local file system + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Create an empty directory which is not a partition directory (lacks partition metadata) + final String nonPartitionDirectory = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0] + "-nonpartition"; + Files.createDirectories(Paths.get(basePath, nonPartitionDirectory)); + + // Create some commits + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + testTable.withPartitionMetaFiles("p1", "p2") + .addCommit("001").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10) + .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10) + .addInflightCommit("003").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + client.startCommitWithTime("005"); + + List partitions = metadataWriter(client).metadata().getAllPartitionPaths(); + assertFalse(partitions.contains(nonPartitionDirectory), + "Must not contain the non-partition " + nonPartitionDirectory); + assertTrue(partitions.contains("p1"), "Must contain partition p1"); + assertTrue(partitions.contains("p2"), "Must contain partition p2"); + + FileStatus[] statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p1")); + assertTrue(statuses.length == 2); + statuses = metadata(client).getAllFilesInPartition(new Path(basePath, "p2")); + assertTrue(statuses.length == 5); + } + } + + /** + * Test various table operations sync to Metadata Table correctly. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testTableOperations(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + + // Write 1 (Bulk insert) + String newCommitTime = "001"; + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + validateMetadata(client); + + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 3 (updates) + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 4 (updates and inserts) + newCommitTime = "004"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "005"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Write 5 (updates and inserts) + newCommitTime = "006"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 5); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "007"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Deletes + newCommitTime = "008"; + records = dataGen.generateDeletes(newCommitTime, 10); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + client.delete(deleteKeys, newCommitTime); + validateMetadata(client); + + // Clean + newCommitTime = "009"; + client.clean(newCommitTime); + validateMetadata(client); + + // Restore + client.restoreToInstant("006"); + validateMetadata(client); + } + } + + /** + * Test rollback of various table operations sync to Metadata Table correctly. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testRollbackOperations(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Write 1 (Bulk insert) + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + Rollback of inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Write 3 (updates) + Rollback of updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 20); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of updates and inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + validateMetadata(client); + } + + // Rollback of Deletes + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateDeletes(newCommitTime, 10); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.delete(deleteKeys, newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + + // Rollback of Clean + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.clean(newCommitTime); + validateMetadata(client); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + + // Rollback of partial commits + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(false).build())) { + // Write updates and inserts + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + + // Marker based rollback of partial commits + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, + getWriteConfigBuilder(false, true, false).withRollbackUsingMarkers(true).build())) { + // Write updates and inserts + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + client.syncTableMetadata(); + validateMetadata(client); + } + } + + /** + * Test when syncing rollback to metadata if the commit being rolled back has not been synced that essentially a no-op occurs to metadata. + * Once explicit sync is called, metadata should match. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testRollbackUnsyncedCommit(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Initialize table with metadata + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // Commit with metadata disabled + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + client.rollback(newCommitTime); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) { + assertFalse(metadata(client).isInSync()); + client.syncTableMetadata(); + validateMetadata(client); + } + } + + /** + * Test sync of table operations. + */ + @ParameterizedTest + @EnumSource(HoodieTableType.class) + @Disabled + public void testSync(HoodieTableType tableType) throws Exception { + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + String newCommitTime; + List records; + List writeStatuses; + + // Initial commits without metadata table enabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateInserts(newCommitTime, 5); + client.startCommitWithTime(newCommitTime); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + } + + // Enable metadata table so it initialized by listing from file system + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 5); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + validateMetadata(client); + assertTrue(metadata(client).isInSync()); + } + + // Various table operations without metadata table enabled + String restoreToInstant; + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 5); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertTrue(metadata(client).isInSync()); + + // updates and inserts + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertTrue(metadata(client).isInSync()); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + assertTrue(metadata(client).isInSync()); + } + + // Savepoint + restoreToInstant = newCommitTime; + if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { + client.savepoint("hoodie", "metadata test"); + assertTrue(metadata(client).isInSync()); + } + + // Deletes + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + records = dataGen.generateDeletes(newCommitTime, 5); + JavaRDD deleteKeys = jsc.parallelize(records, 1).map(r -> r.getKey()); + client.startCommitWithTime(newCommitTime); + client.delete(deleteKeys, newCommitTime); + assertTrue(metadata(client).isInSync()); + + // Clean + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.clean(newCommitTime); + assertTrue(metadata(client).isInSync()); + + // updates + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + assertTrue(metadata(client).isInSync()); + + // insert overwrite to test replacecommit + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime, HoodieTimeline.REPLACE_COMMIT_ACTION); + records = dataGen.generateInserts(newCommitTime, 5); + HoodieWriteResult replaceResult = client.insertOverwrite(jsc.parallelize(records, 1), newCommitTime); + writeStatuses = replaceResult.getWriteStatuses().collect(); + assertNoWriteErrors(writeStatuses); + assertTrue(metadata(client).isInSync()); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details + client.syncTableMetadata(); + } + + // Enable metadata table and ensure it is synced + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + client.restoreToInstant(restoreToInstant); + assertFalse(metadata(client).isInSync()); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + client.syncTableMetadata(); + + validateMetadata(client); + assertTrue(metadata(client).isInSync()); + } + } + + /** + * Instants on Metadata Table should be archived as per config. Metadata Table should be automatically compacted as per config. + */ + @Test + public void testCleaningArchivingAndCompaction() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + final int maxDeltaCommitsBeforeCompaction = 4; + HoodieWriteConfig config = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) + .archiveCommitsWith(6, 8).retainCommits(1) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsBeforeCompaction).build()) + // don't archive the data timeline at all. + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(Integer.MAX_VALUE - 1, Integer.MAX_VALUE) + .retainCommits(1).retainFileVersions(1).withAutoClean(true).withAsyncClean(true).build()) + .build(); + + List records; + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, config)) { + for (int i = 1; i < 10; ++i) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + if (i == 1) { + records = dataGen.generateInserts(newCommitTime, 5); + } else { + records = dataGen.generateUpdates(newCommitTime, 2); + } + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + } + } + + HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath); + HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf, config.getBasePath()); + HoodieActiveTimeline metadataTimeline = metadataMetaClient.getActiveTimeline(); + // check that there are compactions. + assertTrue(metadataTimeline.getCommitTimeline().filterCompletedInstants().countInstants() > 0); + // check that cleaning has, once after each compaction. + assertTrue(metadataTimeline.getCleanerTimeline().filterCompletedInstants().countInstants() > 0); + // ensure archiving has happened + long numDataCompletedInstants = datasetMetaClient.getActiveTimeline().filterCompletedInstants().countInstants(); + long numDeltaCommits = metadataTimeline.getDeltaCommitTimeline().filterCompletedInstants().countInstants(); + assertTrue(numDeltaCommits < numDataCompletedInstants, "Must have less delta commits than total completed instants on data timeline."); + } + + /** + * Test various error scenarios. + */ + @Test + public void testErrorCases() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // TESTCASE: If commit on the metadata table succeeds but fails on the dataset, then on next init the metadata table + // should be rolled back to last valid commit. + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 10); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 5); + writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed + // instant so that only the inflight is left over. + String commitInstantFileName = HoodieTimeline.makeCommitFileName(newCommitTime); + assertTrue(fs.delete(new Path(basePath + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + commitInstantFileName), false)); + } + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true), true)) { + // Start the next commit which will rollback the previous one and also should update the metadata table by + // updating it with HoodieRollbackMetadata. + String newCommitTime = client.startCommit(); + + // Dangling commit but metadata should be valid at this time + validateMetadata(client); + + // Next insert + List records = dataGen.generateInserts(newCommitTime, 5); + List writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + + // Post rollback commit and metadata should be valid + validateMetadata(client); + } + } + + /** + * Test non-partitioned datasets. + */ + @Test + public void testNonPartitioned() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + HoodieTestDataGenerator nonPartitionedGenerator = new HoodieTestDataGenerator(new String[] {""}); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Write 1 (Bulk insert) + String newCommitTime = "001"; + List records = nonPartitionedGenerator.generateInserts(newCommitTime, 10); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + validateMetadata(client); + + List metadataPartitions = metadata(client).getAllPartitionPaths(); + assertTrue(metadataPartitions.contains(""), "Must contain empty partition"); + } + } + + /** + * Test various metrics published by metadata table. + */ + @Test + public void testMetadataMetrics() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfigBuilder(true, true, true).build())) { + // Write + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + Registry metricsRegistry = Registry.getRegistry("HoodieMetadata"); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".count")); + assertTrue(metricsRegistry.getAllCounts().containsKey(HoodieMetadataMetrics.INITIALIZE_STR + ".totalDuration")); + assertEquals(metricsRegistry.getAllCounts().get(HoodieMetadataMetrics.INITIALIZE_STR + ".count"), 1L); + assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.size")); + assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.size")); + assertTrue(metricsRegistry.getAllCounts().containsKey("basefile.count")); + assertTrue(metricsRegistry.getAllCounts().containsKey("logfile.count")); + } + } + + /** + * Test when reading from metadata table which is out of sync with dataset that results are still consistent. + */ + @Test + public void testMetadataOutOfSync() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true)); + + // Enable metadata so table is initialized + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { + // Perform Bulk Insert + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 20); + client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect(); + } + + // Perform commit operations with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // Perform Insert + String newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, 20); + client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Perform Upsert + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 20); + client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + + // Compaction + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "004"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + } + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + + // Perform clean operation with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + // One more commit needed to trigger clean so upsert and compact + String newCommitTime = "005"; + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, 20); + client.upsert(jsc.parallelize(records, 1), newCommitTime).collect(); + + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + newCommitTime = "006"; + client.scheduleCompactionAtInstant(newCommitTime, Option.empty()); + client.compact(newCommitTime); + } + + // Clean + newCommitTime = "007"; + client.clean(newCommitTime); + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + + // Perform restore with metadata disabled + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { + client.restoreToInstant("004"); + } + + assertFalse(metadata(unsyncedClient).isInSync()); + validateMetadata(unsyncedClient); + } + + + /** + * Validate the metadata tables contents to ensure it matches what is on the file system. + */ + private void validateMetadata(SparkRDDWriteClient testClient) throws IOException { + HoodieWriteConfig config = testClient.getConfig(); + + SparkRDDWriteClient client; + if (config.isEmbeddedTimelineServerEnabled()) { + testClient.close(); + client = new SparkRDDWriteClient(testClient.getEngineContext(), testClient.getConfig()); + } else { + client = testClient; + } + + HoodieTableMetadata tableMetadata = metadata(client); + assertNotNull(tableMetadata, "MetadataReader should have been initialized"); + if (!config.useFileListingMetadata()) { + return; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + // Partitions should match + FileSystemBackedTableMetadata fsBackedTableMetadata = new FileSystemBackedTableMetadata(engineContext, + new SerializableConfiguration(hadoopConf), config.getBasePath(), config.shouldAssumeDatePartitioning()); + List fsPartitions = fsBackedTableMetadata.getAllPartitionPaths(); + List metadataPartitions = tableMetadata.getAllPartitionPaths(); + + Collections.sort(fsPartitions); + Collections.sort(metadataPartitions); + + assertEquals(fsPartitions.size(), metadataPartitions.size(), "Partitions should match"); + assertTrue(fsPartitions.equals(metadataPartitions), "Partitions should match"); + + // Files within each partition should match + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable table = HoodieSparkTable.create(config, engineContext); + TableFileSystemView tableView = table.getHoodieView(); + fsPartitions.forEach(partition -> { + try { + Path partitionPath; + if (partition.equals("")) { + // Should be the non-partitioned case + partitionPath = new Path(basePath); + } else { + partitionPath = new Path(basePath, partition); + } + FileStatus[] fsStatuses = FSUtils.getAllDataFilesInPartition(fs, partitionPath); + FileStatus[] metaStatuses = tableMetadata.getAllFilesInPartition(partitionPath); + List fsFileNames = Arrays.stream(fsStatuses) + .map(s -> s.getPath().getName()).collect(Collectors.toList()); + List metadataFilenames = Arrays.stream(metaStatuses) + .map(s -> s.getPath().getName()).collect(Collectors.toList()); + Collections.sort(fsFileNames); + Collections.sort(metadataFilenames); + + // File sizes should be valid + Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getLen() > 0)); + + // Block sizes should be valid + Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getBlockSize() > 0)); + List fsBlockSizes = Arrays.stream(fsStatuses).map(FileStatus::getBlockSize).collect(Collectors.toList()); + Collections.sort(fsBlockSizes); + List metadataBlockSizes = Arrays.stream(metaStatuses).map(FileStatus::getBlockSize).collect(Collectors.toList()); + Collections.sort(metadataBlockSizes); + assertEquals(fsBlockSizes, metadataBlockSizes); + + if ((fsFileNames.size() != metadataFilenames.size()) || (!fsFileNames.equals(metadataFilenames))) { + LOG.info("*** File system listing = " + Arrays.toString(fsFileNames.toArray())); + LOG.info("*** Metadata listing = " + Arrays.toString(metadataFilenames.toArray())); + + for (String fileName : fsFileNames) { + if (!metadataFilenames.contains(fileName)) { + LOG.error(partition + "FsFilename " + fileName + " not found in Meta data"); + } + } + for (String fileName : metadataFilenames) { + if (!fsFileNames.contains(fileName)) { + LOG.error(partition + "Metadata file " + fileName + " not found in original FS"); + } + } + } + + assertEquals(fsFileNames.size(), metadataFilenames.size(), "Files within partition " + partition + " should match"); + assertTrue(fsFileNames.equals(metadataFilenames), "Files within partition " + partition + " should match"); + + // FileSystemView should expose the same data + List fileGroups = tableView.getAllFileGroups(partition).collect(Collectors.toList()); + fileGroups.addAll(tableView.getAllReplacedFileGroups(partition).collect(Collectors.toList())); + + fileGroups.forEach(g -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(g)); + fileGroups.forEach(g -> g.getAllBaseFiles().forEach(b -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(b))); + fileGroups.forEach(g -> g.getAllFileSlices().forEach(s -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(s))); + + long numFiles = fileGroups.stream() + .mapToLong(g -> g.getAllBaseFiles().count() + g.getAllFileSlices().mapToLong(s -> s.getLogFiles().count()).sum()) + .sum(); + assertEquals(metadataFilenames.size(), numFiles); + } catch (IOException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + assertTrue(false, "Exception should not be raised: " + e); + } + }); + + HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(client); + assertNotNull(metadataWriter, "MetadataWriter should have been initialized"); + + // Validate write config for metadata table + HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig(); + assertFalse(metadataWriteConfig.useFileListingMetadata(), "No metadata table for metadata table"); + assertFalse(metadataWriteConfig.getFileListingMetadataVerify(), "No verify for metadata table"); + + // Metadata table should be in sync with the dataset + assertTrue(metadata(client).isInSync()); + HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath); + + // Metadata table is MOR + assertEquals(metadataMetaClient.getTableType(), HoodieTableType.MERGE_ON_READ, "Metadata Table should be MOR"); + + // Metadata table is HFile format + assertEquals(metadataMetaClient.getTableConfig().getBaseFileFormat(), HoodieFileFormat.HFILE, + "Metadata Table base file format should be HFile"); + + // Metadata table has a fixed number of partitions + // Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory + // in the .hoodie folder. + List metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath), + false, false, false); + assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size()); + + // Metadata table should automatically compact and clean + // versions are +1 as autoclean / compaction happens end of commits + int numFileVersions = metadataWriteConfig.getCleanerFileVersionsRetained() + 1; + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metadataMetaClient, metadataMetaClient.getActiveTimeline()); + metadataTablePartitions.forEach(partition -> { + List latestSlices = fsView.getLatestFileSlices(partition).collect(Collectors.toList()); + assertTrue(latestSlices.stream().map(FileSlice::getBaseFile).count() <= 1, "Should have a single latest base file"); + assertTrue(latestSlices.size() <= 1, "Should have a single latest file slice"); + assertTrue(latestSlices.size() <= numFileVersions, "Should limit file slice to " + + numFileVersions + " but was " + latestSlices.size()); + }); + + LOG.info("Validation time=" + timer.endTimer()); + } + + private HoodieBackedTableMetadataWriter metadataWriter(SparkRDDWriteClient client) { + return (HoodieBackedTableMetadataWriter) SparkHoodieBackedTableMetadataWriter + .create(hadoopConf, client.getConfig(), new HoodieSparkEngineContext(jsc)); + } + + private HoodieTableMetadata metadata(SparkRDDWriteClient client) { + HoodieWriteConfig clientConfig = client.getConfig(); + return HoodieTableMetadata.create(client.getEngineContext(), clientConfig.getMetadataConfig(), clientConfig.getBasePath(), + clientConfig.getSpillableMapBasePath()); + } + + // TODO: this can be moved to TestHarness after merge from master + private void assertNoWriteErrors(List statuses) { + // Verify there are no errors + for (WriteStatus status : statuses) { + assertFalse(status.hasErrors(), "Errors found in write of " + status.getFileId()); + } + } + + private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) { + return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build(); + } + + private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) + .withAutoCommit(autoCommit) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) + .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") + .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() + .withEnableBackupForRemoteFileSystemView(false).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(useFileListingMetadata) + .enableReuse(false) + .enableMetrics(enableMetrics) + .enableFallback(false).build()) + .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) + .withExecutorMetrics(true).usePrefix("unit-test").build()); + } + + @Override + protected HoodieTableType getTableType() { + return tableType; + } +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 00f1ea00ea94b..4fff08abfbcef 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -18,6 +18,8 @@ package org.apache.hudi.table; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; @@ -28,6 +30,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.bootstrap.TestBootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BootstrapFileMapping; @@ -38,9 +41,11 @@ import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -57,6 +62,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -65,9 +71,6 @@ import org.apache.hudi.index.SparkHoodieIndex; import org.apache.hudi.table.action.clean.CleanPlanner; import org.apache.hudi.testutils.HoodieClientTestBase; - -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -76,6 +79,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import scala.Tuple3; import java.io.File; import java.io.IOException; @@ -96,8 +100,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import scala.Tuple3; - import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; @@ -523,7 +525,8 @@ private List runCleaner(HoodieWriteConfig config, boolean simul @ValueSource(booleans = {false, true}) public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception { HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) @@ -614,7 +617,8 @@ public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throw public void testKeepLatestFileVersionsMOR() throws Exception { HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); @@ -644,6 +648,153 @@ public void testKeepLatestFileVersionsMOR() throws Exception { assertTrue(testTable.logFileExists(p0, "001", file1P0, 3)); } + /** + * Test HoodieTable.clean() Cleaning by commit logic for MOR table with Log files. + */ + @Test + public void testKeepLatestCommitsMOR() throws Exception { + + HoodieWriteConfig config = + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()) + .build(); + + HoodieTableMetaClient metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + String p0 = "2020/01/01"; + + // Make 3 files, one base file and 2 log files associated with base file + String file1P0 = testTable.addDeltaCommit("000").getFileIdsWithBaseFilesInPartitions(p0).get(p0); + testTable.forDeltaCommit("000") + .withLogFile(p0, file1P0, 1) + .withLogFile(p0, file1P0, 2); + + // Make 2 files, one base file and 1 log files associated with base file + testTable.addDeltaCommit("001") + .withBaseFilesInPartition(p0, file1P0) + .withLogFile(p0, file1P0, 3); + + // Make 2 files, one base file and 1 log files associated with base file + testTable.addDeltaCommit("002") + .withBaseFilesInPartition(p0, file1P0) + .withLogFile(p0, file1P0, 4); + + List hoodieCleanStats = runCleaner(config); + assertEquals(3, + getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() + .size(), "Must clean three files, one parquet and 2 log files"); + assertFalse(testTable.baseFileExists(p0, "000", file1P0)); + assertFalse(testTable.logFilesExist(p0, "000", file1P0, 1, 2)); + assertTrue(testTable.baseFileExists(p0, "001", file1P0)); + assertTrue(testTable.logFileExists(p0, "001", file1P0, 3)); + assertTrue(testTable.baseFileExists(p0, "002", file1P0)); + assertTrue(testTable.logFileExists(p0, "002", file1P0, 4)); + } + + @Test + public void testCleanWithReplaceCommits() throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) + .build(); + + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + String p0 = "2020/01/01"; + String p1 = "2020/01/02"; + + // make 1 commit, with 1 file per partition + String file1P0C0 = UUID.randomUUID().toString(); + String file1P1C0 = UUID.randomUUID().toString(); + testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); + + HoodieCommitMetadata commitMetadata = generateCommitMetadata( + Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }) + ); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000001"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + metaClient = HoodieTableMetaClient.reload(metaClient); + + List hoodieCleanStatsOne = runCleaner(config); + assertEquals(0, hoodieCleanStatsOne.size(), "Must not scan any partitions and clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next replacecommit, with 1 clustering operation. logically delete p0. No change to p1 + Map partitionAndFileId002 = testTable.forReplaceCommit("00000000000002").getFileIdsWithBaseFilesInPartitions(p0); + String file2P0C1 = partitionAndFileId002.get(p0); + testTable.addReplaceCommit("00000000000002", generateReplaceCommitMetadata(p0, file1P0C0, file2P0C1)); + + // run cleaner + List hoodieCleanStatsTwo = runCleaner(config); + assertEquals(0, hoodieCleanStatsTwo.size(), "Must not scan any partitions and clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next replacecommit, with 1 clustering operation. Replace data in p1. No change to p0 + Map partitionAndFileId003 = testTable.forReplaceCommit("00000000000003").getFileIdsWithBaseFilesInPartitions(p1); + String file3P1C2 = partitionAndFileId003.get(p1); + testTable.addReplaceCommit("00000000000003", generateReplaceCommitMetadata(p1, file1P1C0, file3P1C2)); + + // run cleaner + List hoodieCleanStatsThree = runCleaner(config); + assertEquals(0, hoodieCleanStatsThree.size(), "Must not scan any partitions and clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p1, "00000000000003", file3P1C2)); + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next replacecommit, with 1 clustering operation. Replace data in p0 again + Map partitionAndFileId004 = testTable.forReplaceCommit("00000000000004").getFileIdsWithBaseFilesInPartitions(p0); + String file4P0C3 = partitionAndFileId004.get(p0); + testTable.addReplaceCommit("00000000000004", generateReplaceCommitMetadata(p0, file2P0C1, file4P0C3)); + + // run cleaner + List hoodieCleanStatsFour = runCleaner(config); + assertTrue(testTable.baseFileExists(p0, "00000000000004", file4P0C3)); + assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p1, "00000000000003", file3P1C2)); + assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + //file1P1C0 still stays because its not replaced until 3 and its the only version available + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next replacecommit, with 1 clustering operation. Replace all data in p1. no new files created + Map partitionAndFileId005 = testTable.forReplaceCommit("00000000000005").getFileIdsWithBaseFilesInPartitions(p1); + String file4P1C4 = partitionAndFileId005.get(p1); + testTable.addReplaceCommit("00000000000005", generateReplaceCommitMetadata(p1, file3P1C2, file4P1C4)); + + List hoodieCleanStatsFive = runCleaner(config, 2); + assertTrue(testTable.baseFileExists(p0, "00000000000004", file4P0C3)); + assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p1, "00000000000003", file3P1C2)); + assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertFalse(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + } + + private HoodieReplaceCommitMetadata generateReplaceCommitMetadata(String partition, String replacedFileId, String newFileId) { + HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata(); + replaceMetadata.addReplaceFileId(partition, replacedFileId); + replaceMetadata.setOperationType(WriteOperationType.CLUSTER); + if (!StringUtils.isNullOrEmpty(newFileId)) { + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setPartitionPath(partition); + writeStat.setPath(newFileId); + writeStat.setFileId(newFileId); + replaceMetadata.addWriteStat(partition, writeStat); + } + return replaceMetadata; + } + @Test public void testCleanMetadataUpgradeDowngrade() { String instantTime = "000"; @@ -825,7 +976,8 @@ private static Stream argumentsForTestKeepLatestCommits() { @ParameterizedTest @MethodSource("argumentsForTestKeepLatestCommits") public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withIncrementalCleaningMode(enableIncrementalClean) .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) @@ -1000,7 +1152,8 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { */ @Test public void testCleaningWithZeroPartitionPaths() throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1021,7 +1174,8 @@ public void testCleaningWithZeroPartitionPaths() throws Exception { */ @Test public void testKeepLatestCommitsWithPendingCompactions() throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) .build(); @@ -1044,7 +1198,8 @@ public void testKeepLatestCommitsWithPendingCompactions() throws Exception { @ValueSource(booleans = {false, true}) public void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) throws Exception { HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath).withAssumeDatePartitioning(true) + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build()) .build(); @@ -1067,7 +1222,8 @@ public void testKeepLatestVersionsWithPendingCompactions(boolean retryFailure) t public void testCleanPreviousCorruptedCleanFiles() throws IOException { HoodieWriteConfig config = HoodieWriteConfig.newBuilder() - .withPath(basePath).withAssumeDatePartitioning(true) + .withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 8b47fa3d49457..a6ac276e6ebc9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -22,12 +22,14 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; @@ -46,6 +48,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.Transformations; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; @@ -64,7 +68,7 @@ import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.avro.generic.GenericRecord; @@ -147,7 +151,7 @@ public void testSimpleInsertAndUpdate() throws Exception { client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - insertAndGetFilePaths(records, client, cfg, newCommitTime); + insertRecords(records, client, cfg, newCommitTime); /** * Write 2 (updates) @@ -155,7 +159,7 @@ public void testSimpleInsertAndUpdate() throws Exception { newCommitTime = "004"; client.startCommitWithTime(newCommitTime); records = dataGen.generateUpdates(newCommitTime, 100); - updateAndGetFilePaths(records, client, cfg, newCommitTime); + updateRecords(records, client, cfg, newCommitTime); String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); @@ -163,7 +167,6 @@ public void testSimpleInsertAndUpdate() throws Exception { HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable); tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); - HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles); Stream dataFilesToRead = tableView.getLatestBaseFiles(); assertTrue(dataFilesToRead.findAny().isPresent()); @@ -195,7 +198,7 @@ public void testSimpleInsertAndUpdateHFile() throws Exception { client.startCommitWithTime(newCommitTime); List records = dataGen.generateInserts(newCommitTime, 200); - insertAndGetFilePaths(records, client, cfg, newCommitTime); + insertRecords(records, client, cfg, newCommitTime); /** * Write 2 (updates) @@ -203,7 +206,7 @@ public void testSimpleInsertAndUpdateHFile() throws Exception { newCommitTime = "004"; client.startCommitWithTime(newCommitTime); records = dataGen.generateUpdates(newCommitTime, 100); - updateAndGetFilePaths(records, client, cfg, newCommitTime); + updateRecords(records, client, cfg, newCommitTime); String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); @@ -228,6 +231,81 @@ public void testSimpleInsertAndUpdateHFile() throws Exception { } } + @Test + public void testSimpleClusteringNoUpdates() throws Exception { + testClustering(false); + } + + @Test + public void testSimpleClusteringWithUpdates() throws Exception { + testClustering(true); + } + + private void testClustering(boolean doUpdates) throws Exception { + // set low compaction small File Size to generate more file groups. + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + HoodieWriteConfig cfg = getConfigBuilder(true, 10L, clusteringConfig).build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + + /** + * Write 1 (only inserts) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 400); + insertRecords(records.subList(0, 200), client, cfg, newCommitTime); + + /** + * Write 2 (more inserts to create new files) + */ + // we already set small file size to small number to force inserts to go into new file. + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + insertRecords(records.subList(200, 400), client, cfg, newCommitTime); + + if (doUpdates) { + /** + * Write 3 (updates) + */ + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 100); + updateRecords(records, client, cfg, newCommitTime); + } + + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); + FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable); + // expect 2 base files for each partition + assertEquals(dataGen.getPartitionPaths().length * 2, allFiles.length); + + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); + // verify all files are included in clustering plan. + assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); + + // Do the clustering and validate + client.cluster(clusteringCommitTime, true); + + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context, metaClient); + Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) + .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); + // verify there should be only one base file per partition after clustering. + assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); + + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); + assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), + "Expecting a single commit."); + assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); + assertEquals(400, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"), + "Must contain 200 records"); + } + } + // test incremental read does not go past compaction instant for RO views // For RT views, incremental read can go past compaction @Test @@ -244,7 +322,7 @@ public void testIncrementalReadsWithCompaction() throws Exception { client.startCommitWithTime(commitTime1); List records001 = dataGen.generateInserts(commitTime1, 200); - insertAndGetFilePaths(records001, client, cfg, commitTime1); + insertRecords(records001, client, cfg, commitTime1); // verify only one base file shows up with commit time 001 FileStatus[] snapshotROFiles = getROSnapshotFiles(partitionPath); @@ -265,7 +343,7 @@ public void testIncrementalReadsWithCompaction() throws Exception { String updateTime = "004"; client.startCommitWithTime(updateTime); List records004 = dataGen.generateUpdates(updateTime, 100); - updateAndGetFilePaths(records004, client, cfg, updateTime); + updateRecords(records004, client, cfg, updateTime); // verify RO incremental reads - only one parquet file shows up because updates to into log files incrementalROFiles = getROIncrementalFiles(partitionPath, false); @@ -292,7 +370,7 @@ public void testIncrementalReadsWithCompaction() throws Exception { String insertsTime = "006"; List records006 = dataGen.generateInserts(insertsTime, 200); client.startCommitWithTime(insertsTime); - insertAndGetFilePaths(records006, client, cfg, insertsTime); + insertRecords(records006, client, cfg, insertsTime); // verify new write shows up in snapshot mode even though there is pending compaction snapshotROFiles = getROSnapshotFiles(partitionPath); @@ -809,7 +887,7 @@ public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2) - .withAutoCommit(false).withAssumeDatePartitioning(true) + .withAutoCommit(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withEmbeddedTimelineServerEnabled(true) @@ -923,7 +1001,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { // Write them to corresponding avro logfiles metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); - HoodieWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) + HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) .withLogAppends(updatedRecords); // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state ((SyncableFileSystemView) (table.getSliceView())).reset(); @@ -988,11 +1066,16 @@ public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception { long numLogFiles = 0; for (String partitionPath : dataGen.getPartitionPaths()) { - assertEquals(0, tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getBaseFile().isPresent()).count()); - assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); - numLogFiles += tableRTFileSystemView.getLatestFileSlices(partitionPath) - .filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); + List allSlices = tableRTFileSystemView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); + assertEquals(0, allSlices.stream().filter(fileSlice -> fileSlice.getBaseFile().isPresent()).count()); + assertTrue(allSlices.stream().anyMatch(fileSlice -> fileSlice.getLogFiles().count() > 0)); + long logFileCount = allSlices.stream().filter(fileSlice -> fileSlice.getLogFiles().count() > 0).count(); + if (logFileCount > 0) { + // check the log versions start from the base version + assertTrue(allSlices.stream().map(slice -> slice.getLogFiles().findFirst().get().getLogVersion()) + .allMatch(version -> version.equals(HoodieLogFile.LOGFILE_BASE_VERSION))); + } + numLogFiles += logFileCount; } assertTrue(numLogFiles > 0); @@ -1469,26 +1552,37 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieI return getConfigBuilder(autoCommit, false, indexType); } + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, long compactionSmallFileSize, HoodieClusteringConfig clusteringConfig) { + return getConfigBuilder(autoCommit, false, IndexType.BLOOM, compactionSmallFileSize, clusteringConfig); + } + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean rollbackUsingMarkers, HoodieIndex.IndexType indexType) { + return getConfigBuilder(autoCommit, rollbackUsingMarkers, indexType, 1024 * 1024 * 1024L, HoodieClusteringConfig.newBuilder().build()); + } + + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean rollbackUsingMarkers, HoodieIndex.IndexType indexType, + long compactionSmallFileSize, HoodieClusteringConfig clusteringConfig) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2) - .withAutoCommit(autoCommit).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withAutoCommit(autoCommit) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(compactionSmallFileSize) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() .withEnableBackupForRemoteFileSystemView(false).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) + .withClusteringConfig(clusteringConfig) .withRollbackUsingMarkers(rollbackUsingMarkers); } - private FileStatus[] insertAndGetFilePaths(List records, SparkRDDWriteClient client, - HoodieWriteConfig cfg, String commitTime) throws IOException { + private void insertRecords(List records, SparkRDDWriteClient client, + HoodieWriteConfig cfg, String commitTime) throws IOException { JavaRDD writeRecords = jsc.parallelize(records, 1); List statuses = client.insert(writeRecords, commitTime).collect(); assertNoWriteErrors(statuses); + assertFileSizes(statuses); metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath()); HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); @@ -1510,11 +1604,10 @@ private FileStatus[] insertAndGetFilePaths(List records, SparkRDDW dataFilesToRead = roView.getLatestBaseFiles(); assertTrue(dataFilesToRead.findAny().isPresent(), "should list the parquet files we wrote in the delta commit"); - return allFiles; } - private FileStatus[] updateAndGetFilePaths(List records, SparkRDDWriteClient client, - HoodieWriteConfig cfg, String commitTime) throws IOException { + private void updateRecords(List records, SparkRDDWriteClient client, + HoodieWriteConfig cfg, String commitTime) throws IOException { Map recordsMap = new HashMap<>(); for (HoodieRecord rec : records) { if (!recordsMap.containsKey(rec.getKey())) { @@ -1525,6 +1618,8 @@ private FileStatus[] updateAndGetFilePaths(List records, SparkRDDW List statuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect(); // Verify there are no errors assertNoWriteErrors(statuses); + assertFileSizes(statuses); + metaClient = HoodieTableMetaClient.reload(metaClient); Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); @@ -1533,8 +1628,13 @@ private FileStatus[] updateAndGetFilePaths(List records, SparkRDDW Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); - return listAllBaseFilesInPath(hoodieTable); + } + + private void assertFileSizes(List statuses) throws IOException { + for (WriteStatus status: statuses) { + assertEquals(FSUtils.getFileSize(metaClient.getFs(), new Path(metaClient.getBasePath(), status.getStat().getPath())), + status.getStat().getFileSizeInBytes()); + } } private FileStatus[] getROSnapshotFiles(String partitionPath) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index c19427c7f809a..f40a97c0bbadc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -185,6 +185,21 @@ public void testUpsertPartitioner() throws Exception { assertEquals(2, insertBuckets.size(), "Total of 2 insert buckets"); } + @Test + public void testUpsertPartitionerWithRecordsPerBucket() throws Exception { + final String testPartitionPath = "2016/09/26"; + // Inserts + Updates... Check all updates go together & inserts subsplit + UpsertPartitioner partitioner = getUpsertPartitioner(0, 250, 100, 1024, testPartitionPath, false); + List insertBuckets = partitioner.getInsertBuckets(testPartitionPath); + int insertSplitSize = partitioner.config.getCopyOnWriteInsertSplitSize(); + int remainedInsertSize = 250 - 2 * insertSplitSize; + // will assigned 3 insertBuckets. 100, 100, 50 each + assertEquals(3, insertBuckets.size(), "Total of 3 insert buckets"); + assertEquals(0.4, insertBuckets.get(0).getLeft().weight, "insert " + insertSplitSize + " records"); + assertEquals(0.4, insertBuckets.get(1).getLeft().weight, "insert " + insertSplitSize + " records"); + assertEquals(0.2, insertBuckets.get(2).getLeft().weight, "insert " + remainedInsertSize + " records"); + } + @Test public void testPartitionWeight() throws Exception { final String testPartitionPath = "2016/09/26"; @@ -286,8 +301,8 @@ public void testUpsertPartitionerWithSmallInsertHandling() throws Exception { "Bucket 3 is INSERT"); assertEquals(4, insertBuckets.size(), "Total of 4 insert buckets"); - weights = new Double[] { 0.08, 0.31, 0.31, 0.31}; - cumulativeWeights = new Double[] { 0.08, 0.39, 0.69, 1.0}; + weights = new Double[] { 0.08, 0.42, 0.42, 0.08}; + cumulativeWeights = new Double[] { 0.08, 0.5, 0.92, 1.0}; assertInsertBuckets(weights, cumulativeWeights, insertBuckets); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java index 094c0b39069a3..0f6a150747b8c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java @@ -23,6 +23,7 @@ import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -70,7 +71,8 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) { return HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2) - .withAutoCommit(autoCommit).withAssumeDatePartitioning(true) + .withAutoCommit(autoCommit) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder() diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java index fd6bd839cf8f6..08f9283dde020 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -50,7 +51,9 @@ public class TestAsyncCompaction extends CompactionTestBase { private HoodieWriteConfig getConfig(Boolean autoCommit) { - return getConfigBuilder(autoCommit).build(); + return getConfigBuilder(autoCommit) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).validate(true).build()) + .build(); } @Test @@ -85,8 +88,6 @@ public void testRollbackForInflightCompaction() throws Exception { // Reload and rollback inflight compaction metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath()); HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); - // hoodieTable.rollback(jsc, - // new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false); client.rollbackInflightCompaction( new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 2e6cea70ad921..734fcc2969646 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -44,7 +44,7 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.hudi.testutils.HoodieWriteableTestTable; +import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; @@ -155,7 +155,7 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { updatedRecords = ((JavaRDD)index.tagLocation(updatedRecordsRDD, context, table)).collect(); // Write them to corresponding avro logfiles. Also, set the state transition properly. - HoodieWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) + HoodieSparkWriteableTestTable.of(table, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS) .withLogAppends(updatedRecords); metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java index d140b1183aacf..faf7e7d43c0a5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java @@ -20,14 +20,13 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; - -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; import java.text.SimpleDateFormat; @@ -257,10 +256,13 @@ private List createCompactionOperations(HoodieWriteCo HoodieBaseFile df = TestHoodieBaseFile.newDataFile(k); String partitionPath = keyToPartitionMap.get(k); List logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()); + FileSlice slice = new FileSlice(new HoodieFileGroupId(partitionPath, df.getFileId()), df.getCommitTime()); + slice.setBaseFile(df); + logFiles.stream().forEach(f -> slice.addLogFile(f)); operations.add(new HoodieCompactionOperation(df.getCommitTime(), logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(), partitionPath, - config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles), + config.getCompactionStrategy().captureMetrics(config, slice), df.getBootstrapBaseFile().map(BaseFile::getPath).orElse(null)) ); }); @@ -303,10 +305,11 @@ public long getFileSize() { public static class TestHoodieLogFile extends HoodieLogFile { + private static int version = 0; private final long size; public TestHoodieLogFile(long size) { - super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1"); + super("/tmp/.ce481ee7-9e53-4a2e-999-f9e295fa79c0_20180919184844.log." + version++); this.size = size; } @@ -314,11 +317,6 @@ public static HoodieLogFile newLogFile(long size) { return new TestHoodieLogFile(size); } - @Override - public Path getPath() { - return new Path("/tmp/test-log"); - } - @Override public long getFileSize() { return size; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java index 7acff79ef50fb..83e2b0584eed2 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java @@ -18,20 +18,33 @@ package org.apache.hudi.table.action.rollback; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hadoop.fs.FileStatus; +import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import java.util.Arrays; import java.util.List; import java.util.stream.Stream; @@ -40,13 +53,21 @@ public class TestMarkerBasedRollbackStrategy extends HoodieClientTestBase { + private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with listing metadata enable={0}"; + + public static Stream configParams() { + return Arrays.stream(new Boolean[][] {{true}, {false}}).map(Arguments::of); + } + + private HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE; + @BeforeEach public void setUp() throws Exception { initPath(); initSparkContexts(); initFileSystem(); - initMetaClient(); - initDFS(); + initMetaClient(tableType); + initTestDataGenerator(); } @AfterEach @@ -55,7 +76,7 @@ public void tearDown() throws Exception { } @Test - public void testCopyOnWriteRollback() throws Exception { + public void testCopyOnWriteRollbackWithTestTable() throws Exception { // given: wrote some base files and corresponding markers HoodieTestTable testTable = HoodieTestTable.of(metaClient); String f0 = testTable.addRequestedCommit("000") @@ -85,43 +106,78 @@ public void testCopyOnWriteRollback() throws Exception { assertEquals(1, stats.stream().mapToInt(r -> r.getFailedDeleteFiles().size()).sum()); } - @Test - public void testMergeOnReadRollback() throws Exception { - // given: wrote some base + log files and corresponding markers - HoodieTestTable testTable = HoodieTestTable.of(metaClient); - String f2 = testTable.addRequestedDeltaCommit("000") - .getFileIdsWithBaseFilesInPartitions("partA").get("partA"); - String f1 = testTable.addDeltaCommit("001") - .withLogFile("partA", f2) - .getFileIdsWithBaseFilesInPartitions("partB").get("partB"); - String f3 = "f3"; - String f4 = "f4"; - testTable.forDeltaCommit("001") - .withMarkerFile("partB", f1, IOType.CREATE) - .withMarkerFile("partA", f3, IOType.CREATE) - .withMarkerFile("partA", f2, IOType.APPEND) - .withMarkerFile("partB", f4, IOType.APPEND); + @Tag("functional") + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testCopyOnWriteRollback(boolean useFileListingMetadata) throws Exception { + HoodieWriteConfig writeConfig = getConfigBuilder().withRollbackUsingMarkers(true).withAutoCommit(false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(useFileListingMetadata).build()) + .withPath(basePath).build(); + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, writeConfig)) { + // rollback 2nd commit and ensure stats reflect the info. + List stats = testRun(useFileListingMetadata, writeConfig, writeClient); + + assertEquals(3, stats.size()); + for (HoodieRollbackStat stat : stats) { + assertEquals(1, stat.getSuccessDeleteFiles().size()); + assertEquals(0, stat.getFailedDeleteFiles().size()); + assertEquals(0, stat.getCommandBlocksCount().size()); + assertEquals(0, stat.getWrittenLogFileSizeMap().size()); + } + } + } - // when - List stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002") - .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); + @Tag("functional") + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testMergeOnReadRollback(boolean useFileListingMetadata) throws Exception { + // init MERGE_ON_READ_TABLE + tearDown(); + tableType = HoodieTableType.MERGE_ON_READ; + setUp(); + + HoodieWriteConfig writeConfig = getConfigBuilder().withRollbackUsingMarkers(true).withAutoCommit(false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(useFileListingMetadata).build()) + .withPath(basePath).build(); + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, writeConfig)) { + + // rollback 2nd commit and ensure stats reflect the info. + List stats = testRun(useFileListingMetadata, writeConfig, writeClient); + + assertEquals(3, stats.size()); + for (HoodieRollbackStat stat : stats) { + assertEquals(0, stat.getSuccessDeleteFiles().size()); + assertEquals(0, stat.getFailedDeleteFiles().size()); + assertEquals(1, stat.getCommandBlocksCount().size()); + stat.getCommandBlocksCount().forEach((fileStatus, len) -> assertTrue(fileStatus.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension()))); + assertEquals(1, stat.getWrittenLogFileSizeMap().size()); + stat.getWrittenLogFileSizeMap().forEach((fileStatus, len) -> assertTrue(fileStatus.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension()))); + } + } + } - // then: ensure files are deleted, rollback block is appended (even if append does not exist) - assertEquals(2, stats.size()); - // will have the log file - FileStatus[] partBFiles = testTable.listAllFilesInPartition("partB"); - assertEquals(1, partBFiles.length); - assertTrue(partBFiles[0].getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())); - assertTrue(partBFiles[0].getLen() > 0); + private List testRun(boolean useFileListingMetadata, HoodieWriteConfig writeConfig, SparkRDDWriteClient writeClient) { + String newCommitTime = "001"; + writeClient.startCommitWithTime(newCommitTime); - FileStatus[] partAFiles = testTable.listAllFilesInPartition("partA"); - assertEquals(3, partAFiles.length); - assertEquals(2, Stream.of(partAFiles).filter(s -> s.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())).count()); - assertEquals(1, Stream.of(partAFiles).filter(s -> s.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())).filter(f -> f.getLen() > 0).count()); + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD writeStatuses = writeClient.insert(jsc.parallelize(records, 1), newCommitTime); + writeClient.commit(newCommitTime, writeStatuses); - // only partB/f1_001 will be deleted - assertEquals(1, stats.stream().mapToInt(r -> r.getSuccessDeleteFiles().size()).sum()); - // partA/f3_001 is non existent - assertEquals(1, stats.stream().mapToInt(r -> r.getFailedDeleteFiles().size()).sum()); + // Updates + newCommitTime = "002"; + writeClient.startCommitWithTime(newCommitTime); + records = dataGen.generateUniqueUpdates(newCommitTime, 50); + writeStatuses = writeClient.upsert(jsc.parallelize(records, 1), newCommitTime); + writeStatuses.collect(); + + // rollback 2nd commit and ensure stats reflect the info. + return new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(writeConfig, context, metaClient), context, writeConfig, "003") + .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002")); } + } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index b8e02b905b8cb..6a292f5f1f53f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -46,6 +46,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -91,6 +92,11 @@ public void setUp() throws Exception { initDFSMetaClient(); } + @AfterEach + public void cleanUp() throws Exception { + cleanupResources(); + } + @Test public void testLeftOverUpdatedPropFileCleanup() throws IOException { testUpgradeInternal(true, true, HoodieTableType.MERGE_ON_READ); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java index 1020e932b684c..57767d519c398 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java @@ -21,8 +21,8 @@ import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java index b10781e3b8ec9..e6523af462a78 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java @@ -24,11 +24,11 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; -import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.util.Option; @@ -72,7 +72,6 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im protected transient Configuration hadoopConf = null; protected transient SQLContext sqlContext; protected transient FileSystem fs; - protected transient HoodieTestDataGenerator dataGen = null; protected transient ExecutorService executorService; protected transient HoodieTableMetaClient metaClient; protected transient SparkRDDWriteClient writeClient; @@ -206,6 +205,10 @@ protected void cleanupFileSystem() throws IOException { * @throws IOException */ protected void initMetaClient() throws IOException { + initMetaClient(getTableType()); + } + + protected void initMetaClient(HoodieTableType tableType) throws IOException { if (basePath == null) { throw new IllegalStateException("The base path has not been initialized."); } @@ -214,7 +217,7 @@ protected void initMetaClient() throws IOException { throw new IllegalStateException("The Spark context has not been initialized."); } - metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType()); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); } /** @@ -237,24 +240,6 @@ protected void cleanupClients() throws IOException { } } - /** - * Initializes a test data generator which used to generate test datas. - * - */ - protected void initTestDataGenerator() { - dataGen = new HoodieTestDataGenerator(); - } - - /** - * Cleanups test data generator. - * - */ - protected void cleanupTestDataGenerator() { - if (dataGen != null) { - dataGen = null; - } - } - /** * Initializes a distributed file system and base directory. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 307e0686756a8..c91b51b88667e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -147,6 +147,22 @@ public static long countRecordsSince(JavaSparkContext jsc, String basePath, SQLC } } + public static List getLatestBaseFiles(String basePath, FileSystem fs, + String... paths) { + List latestFiles = new ArrayList<>(); + try { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); + for (String path : paths) { + BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); + latestFiles.addAll(fileSystemView.getLatestBaseFiles().collect(Collectors.toList())); + } + } catch (Exception e) { + throw new HoodieException("Error reading hoodie table as a dataframe", e); + } + return latestFiles; + } + /** * Reads the paths under the a hoodie table out as a DataFrame. */ @@ -154,14 +170,9 @@ public static Dataset read(JavaSparkContext jsc, String basePath, SQLContex String... paths) { List filteredPaths = new ArrayList<>(); try { - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); - for (String path : paths) { - BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient, - metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); - List latestFiles = fileSystemView.getLatestBaseFiles().collect(Collectors.toList()); - for (HoodieBaseFile file : latestFiles) { - filteredPaths.add(file.getPath()); - } + List latestFiles = getLatestBaseFiles(basePath, fs, paths); + for (HoodieBaseFile file : latestFiles) { + filteredPaths.add(file.getPath()); } return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()])); } catch (Exception e) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java index 8104ef7744fce..56335511201fa 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java @@ -43,7 +43,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -84,36 +83,32 @@ public static List getRecordsUsingInputFormat(Configuration conf, .map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal())) .collect(Collectors.toList())); - return inputPaths.stream().map(path -> { - setInputPath(jobConf, path); - List records = new ArrayList<>(); - try { - List splits = Arrays.asList(inputFormat.getSplits(jobConf, 1)); - for (InputSplit split : splits) { - RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null); - Object key = recordReader.createKey(); - ArrayWritable writable = (ArrayWritable) recordReader.createValue(); - while (recordReader.next(key, writable)) { - GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema); - // writable returns an array with [field1, field2, _hoodie_commit_time, - // _hoodie_commit_seqno] - Writable[] values = writable.get(); - schema.getFields().stream() - .filter(f -> !projectCols || projectedColumns.contains(f.name())) - .map(f -> Pair.of(projectedSchema.getFields().stream() - .filter(p -> f.name().equals(p.name())).findFirst().get(), f)) - .forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()])); - records.add(newRecord.build()); - } + List records = new ArrayList<>(); + try { + FileInputFormat.setInputPaths(jobConf, String.join(",", inputPaths)); + InputSplit[] splits = inputFormat.getSplits(jobConf, inputPaths.size()); + + for (InputSplit split : splits) { + RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null); + Object key = recordReader.createKey(); + ArrayWritable writable = (ArrayWritable) recordReader.createValue(); + while (recordReader.next(key, writable)) { + GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema); + // writable returns an array with [field1, field2, _hoodie_commit_time, + // _hoodie_commit_seqno] + Writable[] values = writable.get(); + schema.getFields().stream() + .filter(f -> !projectCols || projectedColumns.contains(f.name())) + .map(f -> Pair.of(projectedSchema.getFields().stream() + .filter(p -> f.name().equals(p.name())).findFirst().get(), f)) + .forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()])); + records.add(newRecord.build()); } - } catch (IOException ie) { - ie.printStackTrace(); } - return records; - }).reduce((a, b) -> { - a.addAll(b); - return a; - }).orElse(new ArrayList<>()); + } catch (IOException ie) { + ie.printStackTrace(); + } + return records; } private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List projectedCols) { @@ -156,10 +151,4 @@ private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf configurable.setConf(conf); jobConf.addResource(conf); } - - private static void setInputPath(JobConf jobConf, String inputPath) { - jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath); - jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath); - jobConf.set("map.input.dir", inputPath); - } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java new file mode 100644 index 0000000000000..8e37c92d34a1d --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java @@ -0,0 +1,102 @@ +/* + * 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.testutils; + +import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Arrays; +import java.util.List; +import java.util.UUID; + +public class HoodieSparkWriteableTestTable extends HoodieWriteableTestTable { + private static final Logger LOG = LogManager.getLogger(HoodieSparkWriteableTestTable.class); + + private HoodieSparkWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { + super(basePath, fs, metaClient, schema, filter); + } + + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) { + return new HoodieSparkWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter); + } + + public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) { + BloomFilter filter = BloomFilterFactory + .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name()); + return of(metaClient, schema, filter); + } + + public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema) { + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + return of(metaClient, schema); + } + + public static HoodieSparkWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) { + HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); + return of(metaClient, schema, filter); + } + + @Override + public HoodieSparkWriteableTestTable addCommit(String instantTime) throws Exception { + return (HoodieSparkWriteableTestTable) super.addCommit(instantTime); + } + + @Override + public HoodieSparkWriteableTestTable forCommit(String instantTime) { + return (HoodieSparkWriteableTestTable) super.forCommit(instantTime); + } + + public String getFileIdWithInserts(String partition) throws Exception { + return getFileIdWithInserts(partition, new HoodieRecord[0]); + } + + public String getFileIdWithInserts(String partition, HoodieRecord... records) throws Exception { + return getFileIdWithInserts(partition, Arrays.asList(records)); + } + + public String getFileIdWithInserts(String partition, List records) throws Exception { + String fileId = UUID.randomUUID().toString(); + withInserts(partition, fileId, records); + return fileId; + } + + public HoodieSparkWriteableTestTable withInserts(String partition, String fileId) throws Exception { + return withInserts(partition, fileId, new HoodieRecord[0]); + } + + public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception { + return withInserts(partition, fileId, Arrays.asList(records)); + } + + public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, List records) throws Exception { + super.withInserts(partition, fileId, records, new SparkTaskContextSupplier()); + return this; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java index 04a1712a2197a..3d2019dbdcef9 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java @@ -26,6 +26,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; +import org.apache.spark.package$; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -41,6 +42,8 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.UUID; @@ -139,11 +142,11 @@ public static Row getRandomValue(String partitionPath, boolean isError) { * @param rows Datasets to be converted * @return the List of {@link InternalRow}s thus converted. */ - public static List toInternalRows(Dataset rows, ExpressionEncoder encoder) { + public static List toInternalRows(Dataset rows, ExpressionEncoder encoder) throws Exception { List toReturn = new ArrayList<>(); List rowList = rows.collectAsList(); for (Row row : rowList) { - toReturn.add(encoder.toRow(row).copy()); + toReturn.add(serializeRow(encoder, row).copy()); } return toReturn; } @@ -173,4 +176,17 @@ public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) { .withBulkInsertParallelism(2); } + private static InternalRow serializeRow(ExpressionEncoder encoder, Row row) + throws InvocationTargetException, IllegalAccessException, NoSuchMethodException, ClassNotFoundException { + // TODO remove reflection if Spark 2.x support is dropped + if (package$.MODULE$.SPARK_VERSION().startsWith("2.")) { + Method spark2method = encoder.getClass().getMethod("toRow", Object.class); + return (InternalRow) spark2method.invoke(encoder, row); + } else { + Class serializerClass = Class.forName("org.apache.spark.sql.catalyst.encoders.ExpressionEncoder$Serializer"); + Object serializer = encoder.getClass().getMethod("createSerializer").invoke(encoder); + Method aboveSpark2method = serializerClass.getMethod("apply", Object.class); + return (InternalRow) aboveSpark2method.invoke(serializer, row); + } + } } diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml index cb838390e07ab..0c709274fd39d 100644 --- a/hudi-client/pom.xml +++ b/hudi-client/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 4fbc63da7c644..2981155a277f8 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 @@ -89,6 +89,7 @@ ${basedir}/src/main/avro/HoodieClusteringStrategy.avsc ${basedir}/src/main/avro/HoodieClusteringPlan.avsc ${basedir}/src/main/avro/HoodieRequestedReplaceMetadata.avsc + ${basedir}/src/main/avro/HoodieMetadata.avsc diff --git a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc index fb41f6ef55855..b2444be84aa00 100644 --- a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc +++ b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc @@ -40,6 +40,11 @@ }], "default": null }, + { + "name":"numOutputFileGroups", + "type":["int", "null"], + "default": 1 + }, { "name":"version", "type":["int", "null"], diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc new file mode 100644 index 0000000000000..bf85587a3a7ac --- /dev/null +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieMetadataRecord", + "doc": "A record saved within the Metadata Table", + "fields": [ + { + "name": "key", + "type": "string" + }, + { + "name": "type", + "doc": "Type of the metadata record", + "type": "int" + }, + { "name": "filesystemMetadata", + "doc": "Contains information about partitions and files within the dataset", + "type": ["null", { + "type": "map", + "values": { + "type": "record", + "name": "HoodieMetadataFileInfo", + "fields": [ + { + "name": "size", + "type": "long", + "doc": "Size of the file" + }, + { + "name": "isDeleted", + "type": "boolean", + "doc": "True if this file has been deleted" + } + ] + } + }] + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc index a6bd4c20ef278..a972bfd525e2d 100644 --- a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc @@ -30,11 +30,25 @@ "fields": [ {"name": "partitionPath", "type": "string"}, {"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}}, - {"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}} + {"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}}, + {"name": "rollbackLogFiles", "type": { + "type": "map", + "doc": "Files to which append blocks were written to capture rollback commit", + "values": { + "type": "long", + "doc": "Size of this file in bytes" + } + }}, + {"name": "writtenLogFiles", "type": { + "type": "map", + "doc": "Log files written that were expected to be rolledback", + "values": { + "type": "long", + "doc": "Size of this file in bytes" + } + }} ] - } - } - }, + }}}, { "name":"version", "type":["int", "null"], diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index 3b356a7924d0e..ec19fe354317c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.avro; +import org.apache.avro.specific.SpecificRecordBase; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -58,7 +59,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -292,53 +292,53 @@ public static GenericRecord stitchRecords(GenericRecord left, GenericRecord righ return result; } - /** - * Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the old - * schema. - */ - public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) { - return rewrite(record, getCombinedFieldsToWrite(record.getSchema(), newSchema), newSchema); - } - /** * Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new * schema. + * NOTE: Here, the assumption is that you cannot go from an evolved schema (schema with (N) fields) + * to an older schema (schema with (N-1) fields). All fields present in the older record schema MUST be present in the + * new schema and the default/existing values are carried over. + * This particular method does the following things : + * a) Create a new empty GenericRecord with the new schema. + * b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this + * transformed schema + * c) For SpecificRecord, hoodie_metadata_fields have a special treatment. This is done because for code generated + * avro classes (HoodieMetadataRecord), the avro record is a SpecificBaseRecord type instead of a GenericRecord. + * SpecificBaseRecord throws null pointer exception for record.get(name) if name is not present in the schema of the + * record (which happens when converting a SpecificBaseRecord without hoodie_metadata_fields to a new record with it). + * In this case, we do NOT set the defaults for the hoodie_metadata_fields explicitly, instead, the new record assumes + * the default defined in the avro schema itself. + * TODO: See if we can always pass GenericRecord instead of SpecificBaseRecord in some cases. */ - public static GenericRecord rewriteRecordWithOnlyNewSchemaFields(GenericRecord record, Schema newSchema) { - return rewrite(record, new LinkedHashSet<>(newSchema.getFields()), newSchema); - } - - private static GenericRecord rewrite(GenericRecord record, LinkedHashSet fieldsToWrite, Schema newSchema) { + public static GenericRecord rewriteRecord(GenericRecord oldRecord, Schema newSchema) { GenericRecord newRecord = new GenericData.Record(newSchema); - for (Schema.Field f : fieldsToWrite) { - if (record.get(f.name()) == null) { - if (f.defaultVal() instanceof JsonProperties.Null) { - newRecord.put(f.name(), null); - } else { - newRecord.put(f.name(), f.defaultVal()); - } - } else { - newRecord.put(f.name(), record.get(f.name())); + boolean isSpecificRecord = oldRecord instanceof SpecificRecordBase; + for (Schema.Field f : newSchema.getFields()) { + if (!isSpecificRecord) { + copyOldValueOrSetDefault(oldRecord, newRecord, f); + } else if (!isMetadataField(f.name())) { + copyOldValueOrSetDefault(oldRecord, newRecord, f); } } if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( - "Unable to validate the rewritten record " + record + " against schema " + newSchema); + "Unable to validate the rewritten record " + oldRecord + " against schema " + newSchema); } return newRecord; } - /** - * Generates a super set of fields from both old and new schema. - */ - private static LinkedHashSet getCombinedFieldsToWrite(Schema oldSchema, Schema newSchema) { - LinkedHashSet allFields = new LinkedHashSet<>(oldSchema.getFields()); - for (Schema.Field f : newSchema.getFields()) { - if (!allFields.contains(f) && !isMetadataField(f.name())) { - allFields.add(f); + private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field f) { + // cache the result of oldRecord.get() to save CPU expensive hash lookup + Object fieldValue = oldRecord.get(f.name()); + if (fieldValue == null) { + if (f.defaultVal() instanceof JsonProperties.Null) { + newRecord.put(f.name(), null); + } else { + newRecord.put(f.name(), f.defaultVal()); } + } else { + newRecord.put(f.name(), fieldValue); } - return allFields; } public static byte[] compress(String text) { @@ -428,10 +428,12 @@ public static Object getNestedFieldVal(GenericRecord record, String fieldName, b if (returnNullIfNotFound) { return null; - } else { + } else if (valueNode.getSchema().getField(parts[i]) == null) { 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())); + } else { + throw new HoodieException("The value of " + parts[i] + " can not be null"); } } @@ -442,7 +444,7 @@ public static Object getNestedFieldVal(GenericRecord record, String fieldName, b * @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) { + public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue) { if (fieldSchema == null) { return fieldValue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/HoodieRollbackStat.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodieRollbackStat.java index a3191fa026c84..3e4ee34319c7c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/HoodieRollbackStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodieRollbackStat.java @@ -38,13 +38,16 @@ public class HoodieRollbackStat implements Serializable { private final List failedDeleteFiles; // Count of HoodieLogFile to commandBlocks written for a particular rollback private final Map commandBlocksCount; + // all log files with same base instant as instant to be rolledback + private final Map writtenLogFileSizeMap; public HoodieRollbackStat(String partitionPath, List successDeleteFiles, List failedDeleteFiles, - Map commandBlocksCount) { + Map commandBlocksCount, Map writtenLogFileSizeMap) { this.partitionPath = partitionPath; this.successDeleteFiles = successDeleteFiles; this.failedDeleteFiles = failedDeleteFiles; this.commandBlocksCount = commandBlocksCount; + this.writtenLogFileSizeMap = writtenLogFileSizeMap; } public Map getCommandBlocksCount() { @@ -63,6 +66,10 @@ public List getFailedDeleteFiles() { return failedDeleteFiles; } + public Map getWrittenLogFileSizeMap() { + return writtenLogFileSizeMap; + } + public static HoodieRollbackStat.Builder newBuilder() { return new Builder(); } @@ -75,6 +82,7 @@ public static class Builder { private List successDeleteFiles; private List failedDeleteFiles; private Map commandBlocksCount; + private Map writtenLogFileSizeMap; private String partitionPath; public Builder withDeletedFileResults(Map deletedFiles) { @@ -100,6 +108,11 @@ public Builder withRollbackBlockAppendResults(Map commandBlock return this; } + public Builder withWrittenLogFileSizeMap(Map writtenLogFileSizeMap) { + this.writtenLogFileSizeMap = writtenLogFileSizeMap; + return this; + } + public Builder withPartitionPath(String partitionPath) { this.partitionPath = partitionPath; return this; @@ -115,7 +128,10 @@ public HoodieRollbackStat build() { if (commandBlocksCount == null) { commandBlocksCount = Collections.EMPTY_MAP; } - return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount); + if (writtenLogFileSizeMap == null) { + writtenLogFileSizeMap = Collections.EMPTY_MAP; + } + return new HoodieRollbackStat(partitionPath, successDeleteFiles, failedDeleteFiles, commandBlocksCount, writtenLogFileSizeMap); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java new file mode 100644 index 0000000000000..1ead9c8dc28f7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -0,0 +1,228 @@ +/* + * 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.config; + +import javax.annotation.concurrent.Immutable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Configurations used by the HUDI Metadata Table. + */ +@Immutable +public final class HoodieMetadataConfig extends DefaultHoodieConfig { + + public static final String METADATA_PREFIX = "hoodie.metadata"; + + // Enable the internal Metadata Table which saves file listings + public static final String METADATA_ENABLE_PROP = METADATA_PREFIX + ".enable"; + public static final boolean DEFAULT_METADATA_ENABLE = false; + + // Validate contents of Metadata Table on each access against the actual filesystem + public static final String METADATA_VALIDATE_PROP = METADATA_PREFIX + ".validate"; + public static final boolean DEFAULT_METADATA_VALIDATE = false; + public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false; + + // Enable metrics for internal Metadata Table + public static final String METADATA_METRICS_ENABLE_PROP = METADATA_PREFIX + ".metrics.enable"; + public static final boolean DEFAULT_METADATA_METRICS_ENABLE = false; + + // Parallelism for inserts + public static final String METADATA_INSERT_PARALLELISM_PROP = METADATA_PREFIX + ".insert.parallelism"; + public static final int DEFAULT_METADATA_INSERT_PARALLELISM = 1; + + // Async clean + public static final String METADATA_ASYNC_CLEAN_PROP = METADATA_PREFIX + ".clean.async"; + public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = false; + + // Maximum delta commits before compaction occurs + public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = METADATA_PREFIX + ".compact.max.delta.commits"; + public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = 24; + + // Archival settings + public static final String MIN_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.min.commits"; + public static final int DEFAULT_MIN_COMMITS_TO_KEEP = 20; + public static final String MAX_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.max.commits"; + public static final int DEFAULT_MAX_COMMITS_TO_KEEP = 30; + + // Cleaner commits retained + public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained"; + public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3; + + // Controls whether or no the base file open/log merges are reused per API call + public static final String ENABLE_REUSE_PROP = METADATA_PREFIX + ".reuse.enable"; + public static final String DEFAULT_ENABLE_REUSE = "true"; + + // Controls whether or not, upon failure to fetch from metadata table, should fallback to listing. + public static final String ENABLE_FALLBACK_PROP = METADATA_PREFIX + ".fallback.enable"; + public static final String DEFAULT_ENABLE_FALLBACK = "true"; + + public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning"; + public static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false"; + + public static final String FILE_LISTING_PARALLELISM_PROP = "hoodie.file.listing.parallelism"; + public static final int DEFAULT_FILE_LISTING_PARALLELISM = 1500; + + private HoodieMetadataConfig(Properties props) { + super(props); + } + + public static HoodieMetadataConfig.Builder newBuilder() { + return new Builder(); + } + + public int getFileListingParallelism() { + return Math.max(Integer.parseInt(props.getProperty(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP)), 1); + } + + public Boolean shouldAssumeDatePartitioning() { + return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP)); + } + + public boolean useFileListingMetadata() { + return Boolean.parseBoolean(props.getProperty(METADATA_ENABLE_PROP)); + } + + public boolean enableReuse() { + return Boolean.parseBoolean(props.getProperty(ENABLE_REUSE_PROP)); + } + + public boolean enableFallback() { + return Boolean.parseBoolean(props.getProperty(ENABLE_FALLBACK_PROP)); + } + + public boolean validateFileListingMetadata() { + return Boolean.parseBoolean(props.getProperty(METADATA_VALIDATE_PROP)); + } + + public boolean enableMetrics() { + return Boolean.parseBoolean(props.getProperty(METADATA_METRICS_ENABLE_PROP)); + } + + 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 fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder enable(boolean enable) { + props.setProperty(METADATA_ENABLE_PROP, String.valueOf(enable)); + return this; + } + + public Builder enableMetrics(boolean enableMetrics) { + props.setProperty(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics)); + return this; + } + + public Builder enableReuse(boolean reuse) { + props.setProperty(ENABLE_REUSE_PROP, String.valueOf(reuse)); + return this; + } + + public Builder enableFallback(boolean fallback) { + props.setProperty(ENABLE_FALLBACK_PROP, String.valueOf(fallback)); + return this; + } + + public Builder validate(boolean validate) { + props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate)); + return this; + } + + public Builder withInsertParallelism(int parallelism) { + props.setProperty(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism)); + return this; + } + + public Builder withAsyncClean(boolean asyncClean) { + props.setProperty(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean)); + return this; + } + + public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) { + props.setProperty(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction)); + return this; + } + + public Builder archiveCommitsWith(int minToKeep, int maxToKeep) { + props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep)); + props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep)); + return this; + } + + public Builder retainCommits(int commitsRetained) { + props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained)); + return this; + } + + public Builder withFileListingParallelism(int parallelism) { + props.setProperty(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism)); + return this; + } + + public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) { + props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning)); + return this; + } + + public HoodieMetadataConfig build() { + HoodieMetadataConfig config = new HoodieMetadataConfig(props); + setDefaultOnCondition(props, !props.containsKey(METADATA_ENABLE_PROP), METADATA_ENABLE_PROP, + String.valueOf(DEFAULT_METADATA_ENABLE)); + setDefaultOnCondition(props, !props.containsKey(METADATA_METRICS_ENABLE_PROP), METADATA_METRICS_ENABLE_PROP, + String.valueOf(DEFAULT_METADATA_METRICS_ENABLE)); + setDefaultOnCondition(props, !props.containsKey(METADATA_VALIDATE_PROP), METADATA_VALIDATE_PROP, + String.valueOf(DEFAULT_METADATA_VALIDATE)); + setDefaultOnCondition(props, !props.containsKey(METADATA_INSERT_PARALLELISM_PROP), METADATA_INSERT_PARALLELISM_PROP, + String.valueOf(DEFAULT_METADATA_INSERT_PARALLELISM)); + setDefaultOnCondition(props, !props.containsKey(METADATA_ASYNC_CLEAN_PROP), METADATA_ASYNC_CLEAN_PROP, + String.valueOf(DEFAULT_METADATA_ASYNC_CLEAN)); + setDefaultOnCondition(props, !props.containsKey(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP), + METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS)); + setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP, + String.valueOf(DEFAULT_CLEANER_COMMITS_RETAINED)); + setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP, + String.valueOf(DEFAULT_MAX_COMMITS_TO_KEEP)); + setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP, + String.valueOf(DEFAULT_MIN_COMMITS_TO_KEEP)); + setDefaultOnCondition(props, !props.containsKey(FILE_LISTING_PARALLELISM_PROP), FILE_LISTING_PARALLELISM_PROP, + String.valueOf(DEFAULT_FILE_LISTING_PARALLELISM)); + setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP), + HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); + setDefaultOnCondition(props, !props.containsKey(ENABLE_FALLBACK_PROP), ENABLE_FALLBACK_PROP, + DEFAULT_ENABLE_FALLBACK); + setDefaultOnCondition(props, !props.containsKey(ENABLE_REUSE_PROP), ENABLE_REUSE_PROP, + DEFAULT_ENABLE_REUSE); + return config; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java new file mode 100644 index 0000000000000..4f6de8ba5f3c3 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.config; + +import org.apache.avro.Schema; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * A wrapped Schema which can be serialized. + */ +public class SerializableSchema implements Serializable { + + private transient Schema schema; + + public SerializableSchema() { + } + + public SerializableSchema(Schema schema) { + this.schema = newCopy(schema); + } + + public SerializableSchema(SerializableSchema serializableSchema) { + this(serializableSchema.schema); + } + + public static Schema newCopy(Schema schemaObject) { + return new Schema.Parser().parse(schemaObject.toString()); + } + + public Schema get() { + return schema; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + writeObjectTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + readObjectFrom(in); + } + + // create a public write method for unit test + public void writeObjectTo(ObjectOutputStream out) throws IOException { + // Note: writeUTF cannot support string length > 64K. So use writeObject which has small overhead (relatively). + out.writeObject(schema.toString()); + } + + // create a public read method for unit test + public void readObjectFrom(ObjectInputStream in) throws IOException { + try { + schema = new Schema.Parser().parse(in.readObject().toString()); + } catch (ClassNotFoundException e) { + throw new IOException("unable to parse schema", e); + } + } + + @Override + public String toString() { + return schema.toString(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/EngineProperty.java similarity index 96% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java rename to hudi-common/src/main/java/org/apache/hudi/common/engine/EngineProperty.java index aeaec32747780..df2f3bf6fca76 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/EngineProperty.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.common; +package org.apache.hudi.common.engine; /** * Properties specific to each engine, that can be set/obtained from. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/EngineType.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/EngineType.java new file mode 100644 index 0000000000000..5834fa979c221 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/EngineType.java @@ -0,0 +1,26 @@ +/* + * 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.engine; + +/** + * Hoodie data processing engine. support only Apache Spark and Apache Flink for now. + */ +public enum EngineType { + SPARK, FLINK +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java similarity index 90% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java rename to hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java index 408029126eba8..0128ce52b85ff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieEngineContext.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.hudi.client.common; +package org.apache.hudi.common.engine; import org.apache.hudi.common.config.SerializableConfiguration; -import org.apache.hudi.client.common.function.SerializableConsumer; -import org.apache.hudi.client.common.function.SerializableFunction; -import org.apache.hudi.client.common.function.SerializablePairFunction; +import org.apache.hudi.common.function.SerializableConsumer; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; import java.util.List; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java new file mode 100644 index 0000000000000..e8045670244b2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieLocalEngineContext.java @@ -0,0 +1,90 @@ +/* + * 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.engine; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.function.SerializableConsumer; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.Option; + +import org.apache.hudi.common.util.collection.Pair; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper; +import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; + +/** + * A java based engine context, use this implementation on the query engine integrations if needed. + */ +public final class HoodieLocalEngineContext extends HoodieEngineContext { + + public HoodieLocalEngineContext(Configuration conf) { + this(conf, new LocalTaskContextSupplier()); + } + + public HoodieLocalEngineContext(Configuration conf, TaskContextSupplier taskContextSupplier) { + super(new SerializableConfiguration(conf), taskContextSupplier); + } + + @Override + public List map(List data, SerializableFunction func, int parallelism) { + return data.stream().parallel().map(throwingMapWrapper(func)).collect(toList()); + } + + @Override + public List flatMap(List data, SerializableFunction> func, int parallelism) { + return data.stream().parallel().flatMap(throwingFlatMapWrapper(func)).collect(toList()); + } + + @Override + public void foreach(List data, SerializableConsumer consumer, int parallelism) { + data.stream().forEach(throwingForeachWrapper(consumer)); + } + + @Override + public Map mapToPair(List data, SerializablePairFunction func, Integer parallelism) { + return data.stream().map(throwingMapToPairWrapper(func)).collect( + Collectors.toMap(Pair::getLeft, Pair::getRight, (oldVal, newVal) -> newVal) + ); + } + + @Override + public void setProperty(EngineProperty key, String value) { + // no operation for now + } + + @Override + public Option getProperty(EngineProperty key) { + return Option.empty(); + } + + @Override + public void setJobStatus(String activeModule, String activityDescription) { + // no operation for now + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/engine/LocalTaskContextSupplier.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/LocalTaskContextSupplier.java new file mode 100644 index 0000000000000..0c7ae20e1d5e8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/LocalTaskContextSupplier.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.engine; + +import org.apache.hudi.common.util.Option; + +import java.util.function.Supplier; + +public final class LocalTaskContextSupplier extends TaskContextSupplier { + @Override + public Supplier getPartitionIdSupplier() { + return () -> 0; + } + + @Override + public Supplier getStageIdSupplier() { + return () -> 0; + } + + @Override + public Supplier getAttemptIdSupplier() { + return () -> 0L; + } + + @Override + public Option getProperty(EngineProperty prop) { + return Option.empty(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java b/hudi-common/src/main/java/org/apache/hudi/common/engine/TaskContextSupplier.java similarity index 97% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java rename to hudi-common/src/main/java/org/apache/hudi/common/engine/TaskContextSupplier.java index 3a350d61d65c0..813236c07a842 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/engine/TaskContextSupplier.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.common; +package org.apache.hudi.common.engine; import org.apache.hudi.common.util.Option; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 0ce557348b795..a2b3889829780 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -18,38 +18,43 @@ package org.apache.hudi.common.fs; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; 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.hudi.exception.InvalidHoodiePathException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; - +import java.util.HashSet; import java.util.List; import java.util.Map.Entry; import java.util.Objects; +import java.util.Set; import java.util.UUID; import java.util.function.Function; import java.util.regex.Matcher; @@ -193,8 +198,17 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP /** * Obtain all the partition paths, that are present in this table, denoted by presence of * {@link HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}. + * + * If the basePathStr is a subdirectory of .hoodie folder then we assume that the partitions of an internal + * table (a hoodie table within the .hoodie directory) are to be obtained. + * + * @param fs FileSystem instance + * @param basePathStr base directory */ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr) throws IOException { + // If the basePathStr is a folder within the .hoodie directory then we are listing partitions within an + // internal table. + final boolean isMetadataTable = HoodieTableMetadata.isMetadataTable(basePathStr); final Path basePath = new Path(basePathStr); final List partitions = new ArrayList<>(); processFiles(fs, basePathStr, (locatedFileStatus) -> { @@ -203,7 +217,7 @@ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, Str partitions.add(getRelativePartitionPath(basePath, filePath.getParent())); } return true; - }, true); + }, !isMetadataTable); return partitions; } @@ -240,12 +254,29 @@ public static void processFiles(FileSystem fs, String basePathStr, Function getAllPartitionPaths(FileSystem fs, String basePathStr, boolean assumeDatePartitioning) - throws IOException { - if (assumeDatePartitioning) { - return getAllPartitionFoldersThreeLevelsDown(fs, basePathStr); - } else { - return getAllFoldersWithPartitionMetaFile(fs, basePathStr); + public static List getAllPartitionPaths(HoodieEngineContext engineContext, String basePathStr, + boolean useFileListingFromMetadata, boolean verifyListings, + boolean assumeDatePartitioning) { + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder() + .enable(useFileListingFromMetadata) + .validate(verifyListings) + .withAssumeDatePartitioning(assumeDatePartitioning) + .build(); + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, + FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) { + return tableMetadata.getAllPartitionPaths(); + } catch (Exception e) { + throw new HoodieException("Error fetching partition paths from metadata table", e); + } + } + + public static List getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, + String basePathStr) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr, + FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) { + return tableMetadata.getAllPartitionPaths(); + } catch (Exception e) { + throw new HoodieException("Error fetching partition paths from metadata table", e); } } @@ -384,6 +415,20 @@ public static boolean isLogFile(Path logPath) { return matcher.find() && logPath.getName().contains(".log"); } + /** + * Get the names of all the base and log files in the given partition path. + */ + public static FileStatus[] getAllDataFilesInPartition(FileSystem fs, Path partitionPath) throws IOException { + final Set validFileExtensions = Arrays.stream(HoodieFileFormat.values()) + .map(HoodieFileFormat::getFileExtension).collect(Collectors.toCollection(HashSet::new)); + final String logFileExtension = HoodieFileFormat.HOODIE_LOG.getFileExtension(); + + return Arrays.stream(fs.listStatus(partitionPath, path -> { + String extension = FSUtils.getFileExtension(path.getName()); + return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension); + })).filter(FileStatus::isFile).toArray(FileStatus[]::new); + } + /** * Get the latest log file written from the list of log files passed in. */ @@ -396,10 +441,14 @@ public static Option getLatestLogFile(Stream logFi */ public static Stream getAllLogFiles(FileSystem fs, Path partitionPath, final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException { - return Arrays - .stream(fs.listStatus(partitionPath, - path -> path.getName().startsWith("." + fileId) && path.getName().contains(logFileExtension))) - .map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime)); + try { + return Arrays + .stream(fs.listStatus(partitionPath, + path -> path.getName().startsWith("." + fileId) && path.getName().contains(logFileExtension))) + .map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime)); + } catch (FileNotFoundException e) { + return Stream.builder().build(); + } } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java index c3f6189e8a97e..1faaad5337927 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java @@ -19,6 +19,8 @@ package org.apache.hudi.common.fs; import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -65,15 +67,56 @@ public class HoodieWrapperFileSystem extends FileSystem { public static final String HOODIE_SCHEME_PREFIX = "hoodie-"; - private enum MetricName { - create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles + protected enum MetricName { + create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles, read, write } + private static Registry METRICS_REGISTRY_DATA; + private static Registry METRICS_REGISTRY_META; + + public static void setMetricsRegistry(Registry registry, Registry registryMeta) { + METRICS_REGISTRY_DATA = registry; + METRICS_REGISTRY_META = registryMeta; + } + + private ConcurrentMap openStreams = new ConcurrentHashMap<>(); private FileSystem fileSystem; private URI uri; private ConsistencyGuard consistencyGuard = new NoOpConsistencyGuard(); - private Registry metricsRegistry = Registry.getRegistry(this.getClass().getSimpleName()); + + @FunctionalInterface + public interface CheckedFunction { + R get() throws IOException; + } + + private static Registry getMetricRegistryForPath(Path p) { + return ((p != null) && (p.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME))) + ? METRICS_REGISTRY_META : METRICS_REGISTRY_DATA; + } + + protected static R executeFuncWithTimeMetrics(String metricName, Path p, CheckedFunction func) throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + R res = func.get(); + + Registry registry = getMetricRegistryForPath(p); + if (registry != null) { + registry.increment(metricName); + registry.add(metricName + ".totalDuration", timer.endTimer()); + } + + return res; + } + + protected static R executeFuncWithTimeAndByteMetrics(String metricName, Path p, long byteCount, + CheckedFunction func) throws IOException { + Registry registry = getMetricRegistryForPath(p); + if (registry != null) { + registry.add(metricName + ".totalBytes", byteCount); + } + + return executeFuncWithTimeMetrics(metricName, p, func); + } public HoodieWrapperFileSystem() {} @@ -140,16 +183,17 @@ public URI getUri() { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return fileSystem.open(convertToDefaultPath(f), bufferSize); + return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f), bufferSize)); } @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - final Path translatedPath = convertToDefaultPath(f); - return wrapOutputStream(f, - fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress)); + short replication, long blockSize, Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + final Path translatedPath = convertToDefaultPath(f); + return wrapOutputStream(f, + fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress)); + }); } private FSDataOutputStream wrapOutputStream(final Path path, FSDataOutputStream fsDataOutputStream) @@ -164,79 +208,97 @@ private FSDataOutputStream wrapOutputStream(final Path path, FSDataOutputStream return os; } + private FSDataInputStream wrapInputStream(final Path path, FSDataInputStream fsDataInputStream) throws IOException { + if (fsDataInputStream instanceof TimedFSDataInputStream) { + return fsDataInputStream; + } + return new TimedFSDataInputStream(path, fsDataInputStream); + } + @Override public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite)); + }); } @Override public FSDataOutputStream create(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f))); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f))); + }); } @Override public FSDataOutputStream create(Path f, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress)); + }); } @Override public FSDataOutputStream create(Path f, short replication) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication)); + }); } @Override public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize, - Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress)); + Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress)); + short replication, long blockSize, Progressable progress) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress)); + }); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, - blockSize, progress, checksumOpt)); + short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, + blockSize, progress, checksumOpt)); + }); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { - this.metricsRegistry.increment(MetricName.create.name()); - return wrapOutputStream(f, - fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize)); + return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> { + return wrapOutputStream(f, + fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize)); + }); } @Override @@ -246,50 +308,53 @@ public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) @Override public boolean rename(Path src, Path dst) throws IOException { - this.metricsRegistry.increment(MetricName.rename.name()); - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(src)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + src + " to appear", e); - } - - boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst)); - - if (success) { + return executeFuncWithTimeMetrics(MetricName.rename.name(), src, () -> { try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst)); + consistencyGuard.waitTillFileAppears(convertToDefaultPath(src)); } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + dst + " to appear", e); + throw new HoodieException("Timed out waiting for " + src + " to appear", e); } - try { - consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + src + " to disappear", e); + boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst)); + + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + dst + " to appear", e); + } + + try { + consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + src + " to disappear", e); + } } - } - return success; + return success; + }); } @Override public boolean delete(Path f, boolean recursive) throws IOException { - this.metricsRegistry.increment(MetricName.delete.name()); - boolean success = fileSystem.delete(convertToDefaultPath(f), recursive); - - if (success) { - try { - consistencyGuard.waitTillFileDisappears(f); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for " + f + " to disappear", e); + return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> { + boolean success = fileSystem.delete(convertToDefaultPath(f), recursive); + + if (success) { + try { + consistencyGuard.waitTillFileDisappears(f); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for " + f + " to disappear", e); + } } - } - return success; + return success; + }); } @Override public FileStatus[] listStatus(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertToDefaultPath(f)); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> { + return fileSystem.listStatus(convertToDefaultPath(f)); + }); } @Override @@ -304,27 +369,29 @@ public void setWorkingDirectory(Path newDir) { @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException { - this.metricsRegistry.increment(MetricName.mkdirs.name()); - boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission); - if (success) { - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> { + boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission); + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + } } - } - return success; + return success; + }); } @Override public FileStatus getFileStatus(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.getFileStatus.name()); - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - // pass - } - return fileSystem.getFileStatus(convertToDefaultPath(f)); + return executeFuncWithTimeMetrics(MetricName.getFileStatus.name(), f, () -> { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + // pass + } + return fileSystem.getFileStatus(convertToDefaultPath(f)); + }); } @Override @@ -389,12 +456,12 @@ public Path resolvePath(Path p) throws IOException { @Override public FSDataInputStream open(Path f) throws IOException { - return fileSystem.open(convertToDefaultPath(f)); + return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f))); } @Override public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, short replication, - long blockSize, Progressable progress) throws IOException { + long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize, progress)); @@ -402,7 +469,7 @@ public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int buff @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { + short replication, long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication, blockSize, progress)); @@ -410,7 +477,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, bo @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet flags, - int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { + int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { Path p = convertToDefaultPath(f); return wrapOutputStream(p, fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication, blockSize, progress)); @@ -462,8 +529,9 @@ public boolean setReplication(Path src, short replication) throws IOException { @Override public boolean delete(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.delete.name()); - return delete(f, true); + return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> { + return delete(f, true); + }); } @Override @@ -508,32 +576,37 @@ public RemoteIterator listCorruptFileBlocks(Path path) throws IOException @Override public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertToDefaultPath(f), filter); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> { + return fileSystem.listStatus(convertToDefaultPath(f), filter); + }); } @Override public FileStatus[] listStatus(Path[] files) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertDefaults(files)); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> { + return fileSystem.listStatus(convertDefaults(files)); + }); } @Override public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.listStatus.name()); - return fileSystem.listStatus(convertDefaults(files), filter); + return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> { + return fileSystem.listStatus(convertDefaults(files), filter); + }); } @Override public FileStatus[] globStatus(Path pathPattern) throws IOException { - this.metricsRegistry.increment(MetricName.globStatus.name()); - return fileSystem.globStatus(convertToDefaultPath(pathPattern)); + return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> { + return fileSystem.globStatus(convertToDefaultPath(pathPattern)); + }); } @Override public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { - this.metricsRegistry.increment(MetricName.globStatus.name()); - return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); + return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> { + return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter); + }); } @Override @@ -543,8 +616,9 @@ public RemoteIterator listLocatedStatus(Path f) throws IOExce @Override public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { - this.metricsRegistry.increment(MetricName.listFiles.name()); - return fileSystem.listFiles(convertToDefaultPath(f), recursive); + return executeFuncWithTimeMetrics(MetricName.listFiles.name(), f, () -> { + return fileSystem.listFiles(convertToDefaultPath(f), recursive); + }); } @Override @@ -554,16 +628,17 @@ public Path getHomeDirectory() { @Override public boolean mkdirs(Path f) throws IOException { - this.metricsRegistry.increment(MetricName.mkdirs.name()); - boolean success = fileSystem.mkdirs(convertToDefaultPath(f)); - if (success) { - try { - consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); - } catch (TimeoutException e) { - throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> { + boolean success = fileSystem.mkdirs(convertToDefaultPath(f)); + if (success) { + try { + consistencyGuard.waitTillFileAppears(convertToDefaultPath(f)); + } catch (TimeoutException e) { + throw new HoodieException("Timed out waiting for directory " + f + " to appear", e); + } } - } - return success; + return success; + }); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java index 0b70bedc0988c..6869be80d5742 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareFSDataOutputStream.java @@ -43,7 +43,7 @@ public class SizeAwareFSDataOutputStream extends FSDataOutputStream { private final ConsistencyGuard consistencyGuard; public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out, ConsistencyGuard consistencyGuard, - Runnable closeCallback) throws IOException { + Runnable closeCallback) throws IOException { super(out, null); this.path = path; this.closeCallback = closeCallback; @@ -52,14 +52,22 @@ public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out, Consistenc @Override public synchronized void write(byte[] b, int off, int len) throws IOException { - bytesWritten.addAndGet(len); - super.write(b, off, len); + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path, + len, () -> { + bytesWritten.addAndGet(len); + super.write(b, off, len); + return null; + }); } @Override public void write(byte[] b) throws IOException { - bytesWritten.addAndGet(b.length); - super.write(b); + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path, + b.length, () -> { + bytesWritten.addAndGet(b.length); + super.write(b); + return null; + }); } @Override @@ -76,5 +84,4 @@ public void close() throws IOException { public long getBytesWritten() { return bytesWritten.get(); } - } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java new file mode 100644 index 0000000000000..eca8ec368b869 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/TimedFSDataInputStream.java @@ -0,0 +1,79 @@ +/* + * 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.fs; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.ReadOption; +import org.apache.hadoop.io.ByteBufferPool; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.EnumSet; + +/** + * Wrapper over FSDataInputStream that also times the operations. + */ +public class TimedFSDataInputStream extends FSDataInputStream { + + // Path + private final Path path; + + public TimedFSDataInputStream(Path path, FSDataInputStream in) { + super(in); + this.path = path; + } + + @Override + public int read(ByteBuffer buf) throws IOException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, 0, () -> super.read(buf)); + } + + @Override + public int read(long position, byte[] buffer, int offset, int length) throws IOException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, length, () -> super.read(position, buffer, offset, length)); + } + + @Override + public ByteBuffer read(ByteBufferPool bufferPool, int maxLength, EnumSet opts) + throws IOException, UnsupportedOperationException { + return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, maxLength, () -> super.read(bufferPool, maxLength, opts)); + } + + @Override + public void readFully(long position, byte[] buffer) throws IOException { + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, buffer.length, () -> { + super.readFully(position, buffer); + return null; + }); + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { + HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(), + path, length, () -> { + super.readFully(position, buffer, offset, length); + return null; + }); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java b/hudi-common/src/main/java/org/apache/hudi/common/function/FunctionWrapper.java similarity index 91% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java rename to hudi-common/src/main/java/org/apache/hudi/common/function/FunctionWrapper.java index 4e91bd29d676b..405f57eeedfc5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/FunctionWrapper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/FunctionWrapper.java @@ -16,16 +16,15 @@ * limitations under the License. */ -package org.apache.hudi.client.common.function; +package org.apache.hudi.common.function; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Stream; -import scala.Tuple2; - /** * Function wrapper util class, which catches the exception thrown by input function and return a similar function * with no exception thrown. @@ -62,7 +61,7 @@ public static Consumer throwingForeachWrapper(SerializableConsumer thr }; } - public static Function> throwingMapToPairWrapper(SerializablePairFunction throwingPairFunction) { + public static Function> throwingMapToPairWrapper(SerializablePairFunction throwingPairFunction) { return v1 -> { try { return throwingPairFunction.call(v1); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableConsumer.java similarity index 95% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java rename to hudi-common/src/main/java/org/apache/hudi/common/function/SerializableConsumer.java index d7c420522e605..5448ee164bb32 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableConsumer.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.common.function; +package org.apache.hudi.common.function; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableFunction.java similarity index 95% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java rename to hudi-common/src/main/java/org/apache/hudi/common/function/SerializableFunction.java index d3714bc5b6bd1..7e9a270c622c2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.client.common.function; +package org.apache.hudi.common.function; import java.io.Serializable; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializablePairFunction.java similarity index 88% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java rename to hudi-common/src/main/java/org/apache/hudi/common/function/SerializablePairFunction.java index 155837b7f0112..e3e730b6376c4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializablePairFunction.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package org.apache.hudi.client.common.function; +package org.apache.hudi.common.function; -import scala.Tuple2; +import org.apache.hudi.common.util.collection.Pair; import java.io.Serializable; @@ -27,5 +27,5 @@ */ @FunctionalInterface public interface SerializablePairFunction extends Serializable { - Tuple2 call(I t) throws Exception; + Pair call(I t) throws Exception; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableSupplier.java b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableSupplier.java new file mode 100644 index 0000000000000..0500955f94dc8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/SerializableSupplier.java @@ -0,0 +1,26 @@ +/* + * 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.function; + +import java.io.Serializable; + +@FunctionalInterface +public interface SerializableSupplier extends Serializable { + T get(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java new file mode 100644 index 0000000000000..4fdf9f60dddea --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/LocalRegistry.java @@ -0,0 +1,70 @@ +/* + * 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.metrics; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Registry that tracks metrics local to a single jvm process. + */ +public class LocalRegistry implements Registry { + ConcurrentHashMap counters = new ConcurrentHashMap<>(); + private final String name; + + public LocalRegistry(String name) { + this.name = name; + } + + @Override + public void clear() { + counters.clear(); + } + + @Override + public void increment(String name) { + getCounter(name).increment(); + } + + @Override + public void add(String name, long value) { + getCounter(name).add(value); + } + + /** + * Get all Counter type metrics. + */ + @Override + public Map getAllCounts(boolean prefixWithRegistryName) { + HashMap countersMap = new HashMap<>(); + counters.forEach((k, v) -> { + String key = prefixWithRegistryName ? name + "." + k : k; + countersMap.put(key, v.getValue()); + }); + return countersMap; + } + + private synchronized Counter getCounter(String name) { + if (!counters.containsKey(name)) { + counters.put(name, new Counter()); + } + return counters.get(name); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Metric.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Metric.java index 12b42de23c832..79a7764823b7c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Metric.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Metric.java @@ -18,9 +18,11 @@ package org.apache.hudi.common.metrics; +import java.io.Serializable; + /** * Interface for Hudi Metric Types. */ -public interface Metric { +public interface Metric extends Serializable { Long getValue(); } \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java index 169e8bc9003ca..4ac1e61788972 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/metrics/Registry.java @@ -18,87 +18,99 @@ package org.apache.hudi.common.metrics; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.hudi.common.util.ReflectionUtils; + /** - * Lightweight Metrics Registry to track Hudi events. + * Interface which defines a lightweight Metrics Registry to track Hudi events. */ -public class Registry { - ConcurrentHashMap counters = new ConcurrentHashMap<>(); - final String name; +public interface Registry extends Serializable { - private static ConcurrentHashMap registryMap = new ConcurrentHashMap<>(); + ConcurrentHashMap REGISTRY_MAP = new ConcurrentHashMap<>(); - private Registry(String name) { - this.name = name; + /** + * Get (or create) the registry for a provided name. + * + * This function creates a {@code LocalRegistry}. + * + * @param registryName Name of the registry + */ + static Registry getRegistry(String registryName) { + return getRegistry(registryName, LocalRegistry.class.getName()); } /** - * Get (or create) the registry for a provided name. + * Get (or create) the registry for a provided name and given class. + * + * @param registryName Name of the registry. + * @param clazz The fully qualified name of the registry class to create. */ - public static synchronized Registry getRegistry(String registryName) { - if (!registryMap.containsKey(registryName)) { - registryMap.put(registryName, new Registry(registryName)); + static Registry getRegistry(String registryName, String clazz) { + synchronized (Registry.class) { + if (!REGISTRY_MAP.containsKey(registryName)) { + Registry registry = (Registry)ReflectionUtils.loadClass(clazz, registryName); + REGISTRY_MAP.put(registryName, registry); + } + return REGISTRY_MAP.get(registryName); } - return registryMap.get(registryName); } /** * Get all registered metrics. - * @param flush clean all metrics as part of this operation. + * + * @param flush clear all metrics after this operation. * @param prefixWithRegistryName prefix each metric name with the registry name. * @return */ - public static synchronized Map getAllMetrics(boolean flush, boolean prefixWithRegistryName) { - HashMap allMetrics = new HashMap<>(); - registryMap.forEach((registryName, registry) -> { - allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName)); - if (flush) { - registry.clear(); - } - }); - return allMetrics; - } - - public void clear() { - counters.clear(); + static Map getAllMetrics(boolean flush, boolean prefixWithRegistryName) { + synchronized (Registry.class) { + HashMap allMetrics = new HashMap<>(); + REGISTRY_MAP.forEach((registryName, registry) -> { + allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName)); + if (flush) { + registry.clear(); + } + }); + return allMetrics; + } } - public void increment(String name) { - getCounter(name).increment(); - } + /** + * Clear all metrics. + */ + void clear(); - public void add(String name, long value) { - getCounter(name).add(value); - } + /** + * Increment the metric. + * + * @param name Name of the metric to increment. + */ + void increment(String name); - private synchronized Counter getCounter(String name) { - if (!counters.containsKey(name)) { - counters.put(name, new Counter()); - } - return counters.get(name); - } + /** + * Add value to the metric. + * + * @param name Name of the metric. + * @param value The value to add to the metrics. + */ + void add(String name, long value); /** * Get all Counter type metrics. */ - public Map getAllCounts() { + default Map getAllCounts() { return getAllCounts(false); } /** * Get all Counter type metrics. + * + * @param prefixWithRegistryName If true, the names of all metrics are prefixed with name of this registry. */ - public Map getAllCounts(boolean prefixWithRegistryName) { - HashMap countersMap = new HashMap<>(); - counters.forEach((k, v) -> { - String key = prefixWithRegistryName ? name + "." + k : k; - countersMap.put(key, v.getValue()); - }); - return countersMap; - } - -} \ No newline at end of file + Map getAllCounts(boolean prefixWithRegistryName); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java new file mode 100644 index 0000000000000..3d732fc7fa01a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java @@ -0,0 +1,129 @@ +/* + * 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.hudi.avro.model.HoodieSliceInfo; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * Encapsulates all the needed information about a clustering file slice. This is needed because spark serialization + * does not work with avro objects. + */ +public class ClusteringOperation implements Serializable { + + private String dataFilePath; + private List deltaFilePaths; + private String fileId; + private String partitionPath; + private String bootstrapFilePath; + private int version; + + public static ClusteringOperation create(HoodieSliceInfo sliceInfo) { + return new ClusteringOperation(sliceInfo.getDataFilePath(), new ArrayList<>(sliceInfo.getDeltaFilePaths()), sliceInfo.getFileId(), + sliceInfo.getPartitionPath(), sliceInfo.getBootstrapFilePath(), sliceInfo.getVersion()); + } + + // Only for serialization/de-serialization + @Deprecated + public ClusteringOperation() {} + + private ClusteringOperation(final String dataFilePath, final List deltaFilePaths, final String fileId, + final String partitionPath, final String bootstrapFilePath, final int version) { + this.dataFilePath = dataFilePath; + this.deltaFilePaths = deltaFilePaths; + this.fileId = fileId; + this.partitionPath = partitionPath; + this.bootstrapFilePath = bootstrapFilePath; + this.version = version; + } + + public String getDataFilePath() { + return this.dataFilePath; + } + + public void setDataFilePath(final String dataFilePath) { + this.dataFilePath = dataFilePath; + } + + public List getDeltaFilePaths() { + return this.deltaFilePaths; + } + + public void setDeltaFilePaths(final List deltaFilePaths) { + this.deltaFilePaths = deltaFilePaths; + } + + public String getFileId() { + return this.fileId; + } + + public void setFileId(final String fileId) { + this.fileId = fileId; + } + + public String getPartitionPath() { + return this.partitionPath; + } + + public void setPartitionPath(final String partitionPath) { + this.partitionPath = partitionPath; + } + + public String getBootstrapFilePath() { + return this.bootstrapFilePath; + } + + public void setBootstrapFilePath(final String bootstrapFilePath) { + this.bootstrapFilePath = bootstrapFilePath; + } + + public int getVersion() { + return this.version; + } + + public void setVersion(final int version) { + this.version = version; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ClusteringOperation that = (ClusteringOperation) o; + return getVersion() == that.getVersion() + && Objects.equals(getDataFilePath(), that.getDataFilePath()) + && Objects.equals(getDeltaFilePaths(), that.getDeltaFilePaths()) + && Objects.equals(getFileId(), that.getFileId()) + && Objects.equals(getPartitionPath(), that.getPartitionPath()) + && Objects.equals(getBootstrapFilePath(), that.getBootstrapFilePath()); + } + + @Override + public int hashCode() { + return Objects.hash(getDataFilePath(), getDeltaFilePaths(), getFileId(), getPartitionPath(), getBootstrapFilePath(), getVersion()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java new file mode 100644 index 0000000000000..8fc75a1e77136 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import java.io.IOException; +import java.util.Properties; + +import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro; +import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldVal; + +/** + * {@link HoodieRecordPayload} impl that honors ordering field in both preCombine and combineAndGetUpdateValue. + *

+ * 1. preCombine - Picks the latest delta record for a key, based on an ordering field 2. combineAndGetUpdateValue/getInsertValue - Chooses the latest record based on ordering field value. + */ +public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload { + + public DefaultHoodieRecordPayload(GenericRecord record, Comparable orderingVal) { + super(record, orderingVal); + } + + public DefaultHoodieRecordPayload(Option record) { + this(record.isPresent() ? record.get() : null, (record1) -> 0); // natural order + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException { + if (recordBytes.length == 0) { + return Option.empty(); + } + GenericRecord incomingRecord = bytesToAvro(recordBytes, schema); + /* + * Combining strategy here returns currentValue on disk if incoming record is older. + * The incoming record can be either a delete (sent as an upsert with _hoodie_is_deleted set to true) + * or an insert/update record. In any case, if it is older than the record in disk, the currentValue + * in disk is returned (to be rewritten with new commit time). + * + * NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path + * and need to be dealt with separately. + */ + Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true); + Comparable incomingOrderingVal = (Comparable) getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false); + + // Null check is needed here to support schema evolution. The record in storage may be from old schema where + // the new ordering column might not be present and hence returns null. + if (persistedOrderingVal != null && ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) > 0) { + return Option.of(currentValue); + } + + /* + * We reached a point where the value is disk is older than the incoming record. + * Now check if the incoming record is a delete record. + */ + if (isDeleteRecord(incomingRecord)) { + return Option.empty(); + } else { + return Option.of(incomingRecord); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java index 1b7dcb78d80ba..c97743f4d115e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java @@ -20,6 +20,9 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import java.util.ArrayList; +import java.util.List; + /** * Statistics about a single Hoodie delta log operation. */ @@ -28,6 +31,8 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat { private int logVersion; private long logOffset; + private String baseFile; + private List logFiles = new ArrayList<>(); public void setLogVersion(int logVersion) { this.logVersion = logVersion; @@ -44,4 +49,24 @@ public void setLogOffset(long logOffset) { public long getLogOffset() { return logOffset; } + + public void setBaseFile(String baseFile) { + this.baseFile = baseFile; + } + + public String getBaseFile() { + return baseFile; + } + + public void setLogFiles(List logFiles) { + this.logFiles = logFiles; + } + + public void addLogFiles(String logFile) { + logFiles.add(logFile); + } + + public List getLogFiles() { + return logFiles; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java index fa7f9b1bbe5d7..2515659c7b5fd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java @@ -63,6 +63,12 @@ public HoodieLogFile(Path logPath) { this.fileLen = 0; } + public HoodieLogFile(Path logPath, Long fileLen) { + this.fileStatus = null; + this.pathStr = logPath.toString(); + this.fileLen = fileLen; + } + public HoodieLogFile(String logPathStr) { this.fileStatus = null; this.pathStr = logPathStr; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java new file mode 100644 index 0000000000000..5d71ec3cb8029 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java @@ -0,0 +1,32 @@ +/* + * 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; + +/** + * Holds payload properties that implementation of {@link HoodieRecordPayload} can leverage. + * Since both payload classes and HoodiePayloadConfig needs to access these props, storing it here in hudi-common. + */ +public class HoodiePayloadProps { + + // payload ordering field. This could be used to merge incoming record with that in storage. Implementations of + // {@link HoodieRecordPayload} can leverage if required. + public static final String PAYLOAD_ORDERING_FIELD_PROP = "hoodie.payload.ordering.field"; + public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts"; + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java index 1afdd1b59af64..53fcca16d850e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordPayload.java @@ -29,47 +29,84 @@ import java.io.IOException; import java.io.Serializable; import java.util.Map; +import java.util.Properties; /** - * Every Hoodie table has an implementation of the HoodieRecordPayload This abstracts out callbacks which - * depend on record specific logic. + * Every Hoodie table has an implementation of the HoodieRecordPayload This abstracts out callbacks which depend on record specific logic. */ @PublicAPIClass(maturity = ApiMaturityLevel.STABLE) public interface HoodieRecordPayload extends Serializable { /** - * When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to - * insert/upsert (if combining turned on in HoodieClientConfig). + * This method is deprecated. Please use this {@link #preCombine(HoodieRecordPayload, Properties)} method. */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) T preCombine(T another); /** - * This methods lets you write custom merging/combining logic to produce new values as a function of current value on - * storage and whats contained in this object. + * When more than one HoodieRecord have the same HoodieKey, this function combines them before attempting to insert/upsert by taking in a property map. + * Implementation can leverage the property to decide their business logic to do preCombine. + * @param another instance of another {@link HoodieRecordPayload} to be combined with. + * @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage. + * @return the combined value + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + default T preCombine(T another, Properties properties) { + return preCombine(another); + } + + /** + * This methods is deprecated. Please refer to {@link #combineAndGetUpdateValue(IndexedRecord, Schema, Properties)} for java docs. + */ + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) + Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException; + + /** + * This methods lets you write custom merging/combining logic to produce new values as a function of current value on storage and whats contained + * in this object. Implementations can leverage properties if required. *

- * eg: 1) You are updating counters, you may want to add counts to currentValue and write back updated counts 2) You - * may be reading DB redo logs, and merge them with current image for a database row on storage + * eg: + * 1) You are updating counters, you may want to add counts to currentValue and write back updated counts + * 2) You may be reading DB redo logs, and merge them with current image for a database row on storage + *

* * @param currentValue Current value in storage, to merge/combine this payload with * @param schema Schema used for record + * @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage. * @return new combined/merged value to be written back to storage. EMPTY to skip writing this record. */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException; + default Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException { + return combineAndGetUpdateValue(currentValue, schema); + } /** - * Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. Called when writing a - * new value for the given HoodieKey, wherein there is no existing record in storage to be combined against. (i.e - * insert) Return EMPTY to skip writing this record. + * This method is deprecated. Refer to {@link #getInsertValue(Schema, Properties)} for java docs. + * @param schema Schema used for record + * @return the {@link IndexedRecord} to be inserted. */ - @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + @Deprecated + @PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED) Option getInsertValue(Schema schema) throws IOException; /** - * This method can be used to extract some metadata from HoodieRecordPayload. The metadata is passed to - * {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()} in order to compute some aggregate metrics - * using the metadata in the context of a write success or failure. + * Generates an avro record out of the given HoodieRecordPayload, to be written out to storage. Called when writing a new value for the given + * HoodieKey, wherein there is no existing record in storage to be combined against. (i.e insert) Return EMPTY to skip writing this record. + * Implementations can leverage properties if required. + * @param schema Schema used for record + * @param properties Payload related properties. For example pass the ordering field(s) name to extract from value in storage. + * @return the {@link IndexedRecord} to be inserted. + */ + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + default Option getInsertValue(Schema schema, Properties properties) throws IOException { + return getInsertValue(schema); + } + + /** + * This method can be used to extract some metadata from HoodieRecordPayload. The metadata is passed to {@code WriteStatus.markSuccess()} and + * {@code WriteStatus.markFailure()} in order to compute some aggregate metrics using the metadata in the context of a write success or failure. + * @return the metadata in the form of Map if any. */ @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) default Option> getMetadata() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java index 97288dfe00890..9a640bedee41d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieWriteStat.java @@ -71,7 +71,7 @@ public class HoodieWriteStat implements Serializable { private long numInserts; /** - * Total size of file written. + * Total number of bytes written. */ private long totalWriteBytes; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index f6386b94d53b9..f237156360847 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -38,10 +38,14 @@ public enum WriteOperationType { // delete DELETE("delete"), BOOTSTRAP("bootstrap"), - // insert overwrite + // insert overwrite with static partitioning INSERT_OVERWRITE("insert_overwrite"), // cluster CLUSTER("cluster"), + // delete partition + DELETE_PARTITION("delete_partition"), + // insert overwrite with dynamic partitioning + INSERT_OVERWRITE_TABLE("insert_overwrite_table"), // used for old version UNKNOWN("unknown"); @@ -72,6 +76,12 @@ public static WriteOperationType fromValue(String value) { return DELETE; case "insert_overwrite": return INSERT_OVERWRITE; + case "delete_partition": + return DELETE_PARTITION; + case "insert_overwrite_table": + return INSERT_OVERWRITE_TABLE; + case "cluster": + return CLUSTER; default: throw new HoodieException("Invalid value of Type."); } @@ -88,4 +98,4 @@ public String value() { public static boolean isChangingRecords(WriteOperationType operationType) { return operationType == UPSERT || operationType == UPSERT_PREPPED || operationType == DELETE; } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 372b3936b5b2f..932a455daa217 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -152,7 +152,18 @@ private Schema getTableAvroSchemaFromDataFile() throws Exception { * @throws Exception */ public Schema getTableAvroSchema() throws Exception { - Option schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(true); + return getTableAvroSchema(true); + } + + /** + * Gets schema for a hoodie table in Avro format, can choice if include metadata fields. + * + * @param includeMetadataFields choice if include metadata fields + * @return Avro schema for this table + * @throws Exception + */ + public Schema getTableAvroSchema(boolean includeMetadataFields) throws Exception { + Option schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(includeMetadataFields); return schemaFromCommitMetadata.isPresent() ? schemaFromCommitMetadata.get() : getTableAvroSchemaFromDataFile(); } @@ -296,7 +307,7 @@ public MessageType convertAvroSchemaToParquet(Schema schema) { public static boolean isSchemaCompatible(Schema oldSchema, Schema newSchema) { if (oldSchema.getType() == newSchema.getType() && newSchema.getType() == Schema.Type.RECORD) { // record names must match: - if (!SchemaCompatibility.schemaNameEquals(oldSchema, newSchema)) { + if (!SchemaCompatibility.schemaNameEquals(newSchema, oldSchema)) { return false; } @@ -329,9 +340,11 @@ public static boolean isSchemaCompatible(Schema oldSchema, Schema newSchema) { // All fields in the newSchema record can be populated from the oldSchema record return true; } else { - // Use the checks implemented by + // Use the checks implemented by Avro + // newSchema is the schema which will be used to read the records written earlier using oldSchema. Hence, in the + // check below, use newSchema as the reader schema and oldSchema as the writer schema. org.apache.avro.SchemaCompatibility.SchemaPairCompatibility compatResult = - org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(oldSchema, newSchema); + org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(newSchema, oldSchema); return compatResult.getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java index 8d8ef56da7ab9..6fb0a059094e2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordScanner.java @@ -80,7 +80,7 @@ public abstract class AbstractHoodieLogRecordScanner { // Merge strategy to use when combining records from log private final String payloadClassFQN; // Log File Paths - private final List logFilePaths; + protected final List logFilePaths; // Read Lazily flag private final boolean readBlocksLazily; // Reverse reader - Not implemented yet (NA -> Why do we need ?) @@ -105,7 +105,6 @@ public abstract class AbstractHoodieLogRecordScanner { // Progress private float progress = 0.0f; - // TODO (NA) - Change this to a builder, this constructor is too long public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize) { this.readerSchema = readerSchema; @@ -149,7 +148,8 @@ public void scan() { switch (r.getBlockType()) { case HFILE_DATA_BLOCK: case AVRO_DATA_BLOCK: - LOG.info("Reading a data block from file " + logFile.getPath()); + LOG.info("Reading a data block from file " + logFile.getPath() + " at instant " + + r.getLogBlockHeader().get(INSTANT_TIME)); if (isNewInstantBlock(r) && !readBlocksLazily) { // If this is an avro data block belonging to a different commit/instant, // then merge the last blocks and records into the main result @@ -203,8 +203,7 @@ public void scan() { LOG.info("Rolling back the last corrupted log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); numBlocksRolledBack++; - } else if (lastBlock.getBlockType() != CORRUPT_BLOCK - && targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { + } else if (targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { // rollback last data block or delete block LOG.info("Rolling back the last log block read in " + logFile.getPath()); currentInstantLogBlocks.pop(); @@ -279,12 +278,14 @@ private void processDataBlock(HoodieDataBlock dataBlock) throws Exception { List recs = dataBlock.getRecords(); totalLogRecords.addAndGet(recs.size()); for (IndexedRecord rec : recs) { - HoodieRecord hoodieRecord = - SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); - processNextRecord(hoodieRecord); + processNextRecord(createHoodieRecord(rec)); } } + protected HoodieRecord createHoodieRecord(IndexedRecord rec) { + return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN); + } + /** * Process next record. * @@ -358,4 +359,28 @@ public long getTotalRollbacks() { public long getTotalCorruptBlocks() { return totalCorruptBlocks.get(); } + + /** + * Builder used to build {@code AbstractHoodieLogRecordScanner}. + */ + public abstract static class Builder { + + public abstract Builder withFileSystem(FileSystem fs); + + public abstract Builder withBasePath(String basePath); + + public abstract Builder withLogFilePaths(List logFilePaths); + + public abstract Builder withReaderSchema(Schema schema); + + public abstract Builder withLatestInstantTime(String latestInstantTime); + + public abstract Builder withReadBlocksLazily(boolean readBlocksLazily); + + public abstract Builder withReverseReader(boolean reverseReader); + + public abstract Builder withBufferSize(int bufferSize); + + public abstract AbstractHoodieLogRecordScanner build(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AppendResult.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AppendResult.java new file mode 100644 index 0000000000000..8246edada9161 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AppendResult.java @@ -0,0 +1,50 @@ +/* + * 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.table.log; + +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; + +/** + * Pojo holding information on the result of a {@link org.apache.hudi.common.table.log.HoodieLogFormat.Writer#appendBlock(HoodieLogBlock)}. + */ +public class AppendResult { + + private final HoodieLogFile logFile; + private final long offset; + private final long size; + + public AppendResult(HoodieLogFile logFile, long offset, long size) { + this.logFile = logFile; + this.offset = offset; + this.size = size; + } + + public HoodieLogFile logFile() { + return logFile; + } + + public long offset() { + return offset; + } + + public long size() { + return size; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java new file mode 100644 index 0000000000000..3b73f41cbe539 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -0,0 +1,62 @@ +/* + * 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.table.log; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.io.storage.HoodieFileReader; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice. + */ +public class HoodieFileSliceReader implements Iterator> { + private Iterator> recordsIterator; + + public static HoodieFileSliceReader getFileSliceReader( + HoodieFileReader baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass) throws IOException { + Iterator baseIterator = baseFileReader.getRecordIterator(schema); + while (baseIterator.hasNext()) { + GenericRecord record = (GenericRecord) baseIterator.next(); + HoodieRecord hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass); + scanner.processNextRecord(hoodieRecord); + } + return new HoodieFileSliceReader(scanner.iterator()); + } + + private HoodieFileSliceReader(Iterator> recordsItr) { + this.recordsIterator = recordsItr; + } + + @Override + public boolean hasNext() { + return recordsIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return recordsIterator.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java index 14d523ad9825e..e437b782e2e81 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFileReader.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.log; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.TimedFSDataInputStream; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieCommandBlock; @@ -39,6 +40,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -56,6 +58,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024; // 16 MB + private static final int BLOCK_SCAN_READ_BUFFER_SIZE = 1024 * 1024; // 1 MB private static final Logger LOG = LogManager.getLogger(HoodieLogFileReader.class); private final FSDataInputStream inputStream; @@ -70,11 +73,15 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader { private transient Thread shutdownThread = null; public HoodieLogFileReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, int bufferSize, - boolean readBlockLazily, boolean reverseReader) throws IOException { + boolean readBlockLazily, boolean reverseReader) throws IOException { FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize); - if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) { - this.inputStream = new FSDataInputStream( - new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)); + if (FSUtils.isGCSInputStream(fsDataInputStream)) { + this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream( + new BufferedFSInputStream((FSInputStream) (( + (FSDataInputStream) fsDataInputStream.getWrappedStream()).getWrappedStream()), bufferSize))); + } else if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) { + this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream( + new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize))); } else { // fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream // need to wrap in another BufferedFSInputStream the make bufferSize work? @@ -273,19 +280,25 @@ private boolean isBlockCorrupt(int blocksize) throws IOException { } private long scanForNextAvailableBlockOffset() throws IOException { + // Make buffer large enough to scan through the file as quick as possible especially if it is on S3/GCS. + byte[] dataBuf = new byte[BLOCK_SCAN_READ_BUFFER_SIZE]; + boolean eof = false; while (true) { long currentPos = inputStream.getPos(); try { - boolean hasNextMagic = hasNextMagic(); - if (hasNextMagic) { - return currentPos; - } else { - // No luck - advance and try again - inputStream.seek(currentPos + 1); - } + Arrays.fill(dataBuf, (byte) 0); + inputStream.readFully(dataBuf, 0, dataBuf.length); } catch (EOFException e) { + eof = true; + } + long pos = Bytes.indexOf(dataBuf, HoodieLogFormat.MAGIC); + if (pos >= 0) { + return currentPos + pos; + } + if (eof) { return inputStream.getPos(); } + inputStream.seek(currentPos + dataBuf.length - HoodieLogFormat.MAGIC.length); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java index b3700fbedf0cc..9b643ec6e16c8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormat.java @@ -33,6 +33,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.Iterator; +import java.util.List; /** * File Format for Hoodie Log Files. The File Format consists of blocks each separated with a MAGIC sync marker. A Block @@ -61,14 +62,21 @@ public interface HoodieLogFormat { interface Writer extends Closeable { /** - * @return the path to this {@link HoodieLogFormat} + * @return the path to the current {@link HoodieLogFile} being written to. */ HoodieLogFile getLogFile(); /** - * Append Block returns a new Writer if the log is rolled. + * Append Block to a log file. + * @return {@link AppendResult} containing result of the append. + */ + AppendResult appendBlock(HoodieLogBlock block) throws IOException, InterruptedException; + + /** + * Appends the list of blocks to a logfile. + * @return {@link AppendResult} containing result of the append. */ - Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException; + AppendResult appendBlocks(List blocks) throws IOException, InterruptedException; long getCurrentSize() throws IOException; } @@ -88,7 +96,7 @@ interface Reader extends Closeable, Iterator { * * @return */ - public boolean hasPrev(); + boolean hasPrev(); /** * Read log file in reverse order and return prev block if present. @@ -96,7 +104,7 @@ interface Reader extends Closeable, Iterator { * @return * @throws IOException */ - public HoodieLogBlock prev() throws IOException; + HoodieLogBlock prev() throws IOException; } /** @@ -125,6 +133,8 @@ class WriterBuilder { // version number for this log file. If not specified, then the current version will be // computed by inspecting the file system private Integer logVersion; + // file len of this log file + private Long fileLen = 0L; // Location of the directory containing the log private Path parentPath; // Log File Write Token @@ -142,13 +152,13 @@ public WriterBuilder withReplication(short replication) { return this; } - public WriterBuilder withLogWriteToken(String writeToken) { - this.logWriteToken = writeToken; + public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) { + this.rolloverLogWriteToken = rolloverLogWriteToken; return this; } - public WriterBuilder withRolloverLogWriteToken(String rolloverLogWriteToken) { - this.rolloverLogWriteToken = rolloverLogWriteToken; + public WriterBuilder withLogWriteToken(String logWriteToken) { + this.logWriteToken = logWriteToken; return this; } @@ -182,12 +192,17 @@ public WriterBuilder withLogVersion(int version) { return this; } + public WriterBuilder withFileSize(long fileLen) { + this.fileLen = fileLen; + return this; + } + public WriterBuilder onParentPath(Path parentPath) { this.parentPath = parentPath; return this; } - public Writer build() throws IOException, InterruptedException { + public Writer build() throws IOException { LOG.info("Building HoodieLogFormat Writer"); if (fs == null) { throw new IllegalArgumentException("fs is not specified"); @@ -229,13 +244,14 @@ public Writer build() throws IOException, InterruptedException { if (logWriteToken == null) { // This is the case where we have existing log-file with old format. rollover to avoid any conflicts logVersion += 1; + fileLen = 0L; logWriteToken = rolloverLogWriteToken; } Path logPath = new Path(parentPath, FSUtils.makeLogFileName(logFileId, fileExtension, instantTime, logVersion, logWriteToken)); LOG.info("HoodieLogFile on path " + logPath); - HoodieLogFile logFile = new HoodieLogFile(logPath); + HoodieLogFile logFile = new HoodieLogFile(logPath, fileLen); if (bufferSize == null) { bufferSize = FSUtils.getDefaultBufferSize(fs); @@ -246,8 +262,7 @@ public Writer build() throws IOException, InterruptedException { if (sizeThreshold == null) { sizeThreshold = DEFAULT_SIZE_THRESHOLD; } - return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, logWriteToken, - rolloverLogWriteToken); + return new HoodieLogFormatWriter(fs, logFile, bufferSize, replication, sizeThreshold, rolloverLogWriteToken); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java index 7fe21e9b2955d..d7e4f7ef108bc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieLogFormatWriter.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.HoodieLogFormat.WriterBuilder; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.exception.HoodieException; @@ -38,6 +37,8 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.Collections; +import java.util.List; /** * HoodieLogFormatWriter can be used to append blocks to a log file Use HoodieLogFormat.WriterBuilder to construct. @@ -47,13 +48,13 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { private static final Logger LOG = LogManager.getLogger(HoodieLogFormatWriter.class); private HoodieLogFile logFile; + private FSDataOutputStream output; + private final FileSystem fs; private final long sizeThreshold; private final Integer bufferSize; private final Short replication; - private final String logWriteToken; private final String rolloverLogWriteToken; - private FSDataOutputStream output; private boolean closed = false; private transient Thread shutdownThread = null; @@ -66,14 +67,12 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer { * @param replication * @param sizeThreshold */ - HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold, - String logWriteToken, String rolloverLogWriteToken) { + HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize, Short replication, Long sizeThreshold, String rolloverLogWriteToken) { this.fs = fs; this.logFile = logFile; this.sizeThreshold = sizeThreshold; this.bufferSize = bufferSize; this.replication = replication; - this.logWriteToken = logWriteToken; this.rolloverLogWriteToken = rolloverLogWriteToken; addShutDownHook(); } @@ -105,6 +104,7 @@ private FSDataOutputStream getOutputStream() throws IOException, InterruptedExce if (isAppendSupported) { LOG.info(logFile + " exists. Appending to existing file"); try { + // open the path for append and record the offset this.output = fs.append(path, bufferSize); } catch (RemoteException e) { LOG.warn("Remote Exception, attempting to handle or recover lease", e); @@ -124,9 +124,9 @@ private FSDataOutputStream getOutputStream() throws IOException, InterruptedExce } } if (!isAppendSupported) { - this.logFile = logFile.rollOver(fs, rolloverLogWriteToken); - LOG.info("Append not supported.. Rolling over to " + logFile); + rollOver(); createNewFile(); + LOG.info("Append not supported.. Rolling over to " + logFile); } } else { LOG.info(logFile + " does not exist. Create a new file"); @@ -138,52 +138,66 @@ private FSDataOutputStream getOutputStream() throws IOException, InterruptedExce } @Override - public Writer appendBlock(HoodieLogBlock block) throws IOException, InterruptedException { + public AppendResult appendBlock(HoodieLogBlock block) throws IOException, InterruptedException { + return appendBlocks(Collections.singletonList(block)); + } + @Override + public AppendResult appendBlocks(List blocks) throws IOException, InterruptedException { // Find current version HoodieLogFormat.LogFormatVersion currentLogFormatVersion = new HoodieLogFormatVersion(HoodieLogFormat.CURRENT_VERSION); FSDataOutputStream outputStream = getOutputStream(); - long currentSize = outputStream.size(); - - // 1. Write the magic header for the start of the block - outputStream.write(HoodieLogFormat.MAGIC); - - // bytes for header - byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); - // content bytes - byte[] content = block.getContentBytes(); - // bytes for footer - byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); - - // 2. Write the total size of the block (excluding Magic) - outputStream.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); - - // 3. Write the version of this log block - outputStream.writeInt(currentLogFormatVersion.getVersion()); - // 4. Write the block type - outputStream.writeInt(block.getBlockType().ordinal()); - - // 5. Write the headers for the log block - outputStream.write(headerBytes); - // 6. Write the size of the content block - outputStream.writeLong(content.length); - // 7. Write the contents of the data block - outputStream.write(content); - // 8. Write the footers for the log block - outputStream.write(footerBytes); - // 9. Write the total size of the log block (including magic) which is everything written - // until now (for reverse pointer) - // Update: this information is now used in determining if a block is corrupt by comparing to the - // block size in header. This change assumes that the block size will be the last data written - // to a block. Read will break if any data is written past this point for a block. - outputStream.writeLong(outputStream.size() - currentSize); - // Flush every block to disk + long startPos = outputStream.getPos(); + long sizeWritten = 0; + + for (HoodieLogBlock block: blocks) { + long startSize = outputStream.size(); + + // 1. Write the magic header for the start of the block + outputStream.write(HoodieLogFormat.MAGIC); + + // bytes for header + byte[] headerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockHeader()); + // content bytes + byte[] content = block.getContentBytes(); + // bytes for footer + byte[] footerBytes = HoodieLogBlock.getLogMetadataBytes(block.getLogBlockFooter()); + + // 2. Write the total size of the block (excluding Magic) + outputStream.writeLong(getLogBlockLength(content.length, headerBytes.length, footerBytes.length)); + + // 3. Write the version of this log block + outputStream.writeInt(currentLogFormatVersion.getVersion()); + // 4. Write the block type + outputStream.writeInt(block.getBlockType().ordinal()); + + // 5. Write the headers for the log block + outputStream.write(headerBytes); + // 6. Write the size of the content block + outputStream.writeLong(content.length); + // 7. Write the contents of the data block + outputStream.write(content); + // 8. Write the footers for the log block + outputStream.write(footerBytes); + // 9. Write the total size of the log block (including magic) which is everything written + // until now (for reverse pointer) + // Update: this information is now used in determining if a block is corrupt by comparing to the + // block size in header. This change assumes that the block size will be the last data written + // to a block. Read will break if any data is written past this point for a block. + outputStream.writeLong(outputStream.size() - startSize); + + // Fetch the size again, so it accounts also (9). + sizeWritten += outputStream.size() - startSize; + } + // Flush all blocks to disk flush(); + AppendResult result = new AppendResult(logFile, startPos, sizeWritten); // roll over if size is past the threshold - return rolloverIfNeeded(); + rolloverIfNeeded(); + return result; } /** @@ -201,20 +215,19 @@ private int getLogBlockLength(int contentLength, int headerLength, int footerLen Long.BYTES; // bytes to write totalLogBlockLength at end of block (for reverse ptr) } - private Writer rolloverIfNeeded() throws IOException, InterruptedException { + private void rolloverIfNeeded() throws IOException { // Roll over if the size is past the threshold if (getCurrentSize() > sizeThreshold) { - // TODO - make an end marker which seals the old log file (no more appends possible to that - // file). LOG.info("CurrentSize " + getCurrentSize() + " has reached threshold " + sizeThreshold + ". Rolling over to the next version"); - HoodieLogFile newLogFile = logFile.rollOver(fs, rolloverLogWriteToken); - // close this writer and return the new writer - close(); - return new HoodieLogFormatWriter(fs, newLogFile, bufferSize, replication, sizeThreshold, logWriteToken, - rolloverLogWriteToken); + rollOver(); } - return this; + } + + private void rollOver() throws IOException { + closeStream(); + this.logFile = logFile.rollOver(fs, rolloverLogWriteToken); + this.closed = false; } private void createNewFile() throws IOException { @@ -292,13 +305,12 @@ private void handleAppendExceptionOrRecoverLease(Path path, RemoteException e) // appended to, then the NN will throw an exception saying that it couldn't find any active replica with the // last block. Find more information here : https://issues.apache.org/jira/browse/HDFS-6325 LOG.warn("Failed to open an append stream to the log file. Opening a new log file..", e); - // Rollover the current log file (since cannot get a stream handle) and create new one - this.logFile = logFile.rollOver(fs, rolloverLogWriteToken); + rollOver(); createNewFile(); } else if (e.getClassName().contentEquals(AlreadyBeingCreatedException.class.getName())) { LOG.warn("Another task executor writing to the same log file(" + logFile + ". Rolling over"); // Rollover the current log file (since cannot get a stream handle) and create new one - this.logFile = logFile.rollOver(fs, rolloverLogWriteToken); + rollOver(); createNewFile(); } else if (e.getClassName().contentEquals(RecoveryInProgressException.class.getName()) && (fs instanceof DistributedFileSystem)) { @@ -311,8 +323,9 @@ private void handleAppendExceptionOrRecoverLease(Path path, RemoteException e) // try again this.output = fs.append(path, bufferSize); } else { - LOG.warn("Failed to recover lease on path " + path); - throw new HoodieException(e); + final String msg = "Failed to recover lease on path " + path; + LOG.warn(msg); + throw new HoodieException(msg, e); } } else { // When fs.append() has failed and an exception is thrown, by closing the output stream @@ -320,16 +333,16 @@ private void handleAppendExceptionOrRecoverLease(Path path, RemoteException e) // new attemptId, say taskId.1) it will be able to acquire lease on the log file (as output stream was // closed properly by taskId.0). // - // If close() call were to fail throwing an exception, our best bet is to rollover to a new log file. + // If closeStream() call were to fail throwing an exception, our best bet is to rollover to a new log file. try { - close(); + closeStream(); // output stream has been successfully closed and lease on the log file has been released, // before throwing an exception for the append failure. throw new HoodieIOException("Failed to append to the output stream ", e); } catch (Exception ce) { LOG.warn("Failed to close the output stream for " + fs.getClass().getName() + " on path " + path + ". Rolling over to a new log file."); - this.logFile = logFile.rollOver(fs, rolloverLogWriteToken); + rollOver(); createNewFile(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 025ae91758df3..177be8b919a56 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -57,39 +57,56 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner private static final Logger LOG = LogManager.getLogger(HoodieMergedLogRecordScanner.class); // Final map of compacted/merged records - private final ExternalSpillableMap> records; + protected final ExternalSpillableMap> records; // count of merged records in log private long numMergedRecordsInLog; + private long maxMemorySizeInBytes; // Stores the total time taken to perform reading and merging of log blocks - private final long totalTimeTakenToReadAndMergeBlocks; + private long totalTimeTakenToReadAndMergeBlocks; // A timer for calculating elapsed time in millis public final HoodieTimer timer = new HoodieTimer(); @SuppressWarnings("unchecked") public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, - String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, boolean reverseReader, - int bufferSize, String spillableMapBasePath) { + String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, String spillableMapBasePath) { + this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, + reverseReader, bufferSize, spillableMapBasePath, true); + } + + @SuppressWarnings("unchecked") + public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, + String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, + boolean reverseReader, int bufferSize, String spillableMapBasePath, boolean autoScan) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize); try { // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema)); - // Do the scan and merge - timer.startTimer(); - scan(); - this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); - this.numMergedRecordsInLog = records.size(); - LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); - LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); - LOG.info( - "Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); - LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } catch (IOException e) { - throw new HoodieIOException("IOException when reading log file ", e); + throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e); } + + if (autoScan) { + performScan(); + } + } + + protected void performScan() { + // Do the scan and merge + timer.startTimer(); + scan(); + this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer(); + this.numMergedRecordsInLog = records.size(); + LOG.info("Number of log files scanned => " + logFilePaths.size()); + LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes); + LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); + LOG.info( + "Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); + LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); + LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } @Override @@ -105,6 +122,13 @@ public long getNumMergedRecordsInLog() { return numMergedRecordsInLog; } + /** + * Returns the builder for {@code HoodieMergedLogRecordScanner}. + */ + public static HoodieMergedLogRecordScanner.Builder newBuilder() { + return new Builder(); + } + @Override protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { String key = hoodieRecord.getRecordKey(); @@ -128,5 +152,85 @@ protected void processNextDeletedKey(HoodieKey hoodieKey) { public long getTotalTimeTakenToReadAndMergeBlocks() { return totalTimeTakenToReadAndMergeBlocks; } + + public void close() { + if (records != null) { + records.close(); + } + } + + /** + * Builder used to build {@code HoodieUnMergedLogRecordScanner}. + */ + public static class Builder extends AbstractHoodieLogRecordScanner.Builder { + private FileSystem fs; + private String basePath; + private List logFilePaths; + private Schema readerSchema; + private String latestInstantTime; + private boolean readBlocksLazily; + private boolean reverseReader; + private int bufferSize; + // specific configurations + private Long maxMemorySizeInBytes; + private String spillableMapBasePath; + + public Builder withFileSystem(FileSystem fs) { + this.fs = fs; + return this; + } + + public Builder withBasePath(String basePath) { + this.basePath = basePath; + return this; + } + + public Builder withLogFilePaths(List logFilePaths) { + this.logFilePaths = logFilePaths; + return this; + } + + public Builder withReaderSchema(Schema schema) { + this.readerSchema = schema; + return this; + } + + public Builder withLatestInstantTime(String latestInstantTime) { + this.latestInstantTime = latestInstantTime; + return this; + } + + public Builder withReadBlocksLazily(boolean readBlocksLazily) { + this.readBlocksLazily = readBlocksLazily; + return this; + } + + public Builder withReverseReader(boolean reverseReader) { + this.reverseReader = reverseReader; + return this; + } + + public Builder withBufferSize(int bufferSize) { + this.bufferSize = bufferSize; + return this; + } + + public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) { + this.maxMemorySizeInBytes = maxMemorySizeInBytes; + return this; + } + + public Builder withSpillableMapBasePath(String spillableMapBasePath) { + this.spillableMapBasePath = spillableMapBasePath; + return this; + } + + @Override + public HoodieMergedLogRecordScanner build() { + return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, + latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, + bufferSize, spillableMapBasePath); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java index 9c9df12f0e787..1aac6330e06ba 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -41,6 +41,13 @@ public HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List hoodieRecord) throws Exception { // Just call callback without merging @@ -60,4 +67,71 @@ public static interface LogRecordScannerCallback { public void apply(HoodieRecord record) throws Exception; } + + /** + * Builder used to build {@code HoodieUnMergedLogRecordScanner}. + */ + public static class Builder extends AbstractHoodieLogRecordScanner.Builder { + private FileSystem fs; + private String basePath; + private List logFilePaths; + private Schema readerSchema; + private String latestInstantTime; + private boolean readBlocksLazily; + private boolean reverseReader; + private int bufferSize; + // specific configurations + private LogRecordScannerCallback callback; + + public Builder withFileSystem(FileSystem fs) { + this.fs = fs; + return this; + } + + public Builder withBasePath(String basePath) { + this.basePath = basePath; + return this; + } + + public Builder withLogFilePaths(List logFilePaths) { + this.logFilePaths = logFilePaths; + return this; + } + + public Builder withReaderSchema(Schema schema) { + this.readerSchema = schema; + return this; + } + + public Builder withLatestInstantTime(String latestInstantTime) { + this.latestInstantTime = latestInstantTime; + return this; + } + + public Builder withReadBlocksLazily(boolean readBlocksLazily) { + this.readBlocksLazily = readBlocksLazily; + return this; + } + + public Builder withReverseReader(boolean reverseReader) { + this.reverseReader = reverseReader; + return this; + } + + public Builder withBufferSize(int bufferSize) { + this.bufferSize = bufferSize; + return this; + } + + public Builder withLogRecordScannerCallback(LogRecordScannerCallback callback) { + this.callback = callback; + return this; + } + + @Override + public HoodieUnMergedLogRecordScanner build() { + return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, + latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 61d9b7f233bee..6d2682a4ffa09 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.storage.HoodieHFileReader; import org.apache.log4j.LogManager; @@ -118,6 +119,8 @@ protected byte[] serializeRecords() throws IOException { recordKey = record.get(keyField.pos()).toString(); } byte[] recordBytes = HoodieAvroUtils.indexedRecordToBytes(record); + ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey), + "Writing multiple records with same key not supported for " + this.getClass().getName()); sortedRecordsMap.put(recordKey, recordBytes); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 99fd793e334c6..fcb4fd9176dda 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -62,12 +62,16 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss"); public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( - COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, - INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, - INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, - INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, - REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION)); - + COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, + DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, + SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, + INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, + INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION + )); + private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class); protected HoodieTableMetaClient metaClient; private static AtomicReference lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE)); @@ -306,7 +310,23 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI } /** - * Transition Clean State from inflight to Committed. + * Transition replace requested file to replace inflight. + * + * @param requestedInstant Requested instant + * @param data Extra Metadata + * @return inflight instant + */ + public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.getTimestamp()); + // Then write to timeline + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + /** + * Transition replace inflight to Committed. * * @param inflightInstant Inflight instant * @param data Extra Metadata @@ -321,6 +341,26 @@ public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightI return commitInstant; } + /** + * Revert replace requested State from inflight to requested. + * + * @param inflightInstant Inflight Instant + * @return requested instant + */ + public HoodieInstant revertReplaceCommitInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { transitionState(fromInstant, toInstant, data, false); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index ff251e3144487..ada964fbb9a59 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -109,7 +109,7 @@ public interface HoodieTimeline extends Serializable { /** * Filter this timeline to just include the in-flights excluding compaction instants. * - * @return New instance of HoodieTimeline with just in-flights excluding compaction inflights + * @return New instance of HoodieTimeline with just in-flights excluding compaction instants */ HoodieTimeline filterPendingExcludingCompaction(); @@ -299,6 +299,14 @@ static HoodieInstant getCompactionInflightInstant(final String timestamp) { return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp); } + static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp); + } + + static HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { + return new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, timestamp); + } + /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names * between inflight and completed instants (compaction <=> commit). diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index 640d4894feb69..9b419caed3666 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -18,19 +18,11 @@ package org.apache.hudi.common.table.timeline; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.file.FileReader; -import org.apache.avro.file.SeekableByteArrayInput; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DatumWriter; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; +import org.apache.hudi.avro.model.HoodieReplaceCommitMetadata; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -41,6 +33,17 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.file.FileReader; +import org.apache.avro.file.SeekableByteArrayInput; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecordBase; +import org.apache.hadoop.fs.FileStatus; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Collections; @@ -68,8 +71,12 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac Map partitionMetadataBuilder = new HashMap<>(); int totalDeleted = 0; for (HoodieRollbackStat stat : rollbackStats) { + Map rollbackLogFiles = stat.getCommandBlocksCount().keySet().stream() + .collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen)); + Map probableLogFiles = stat.getWrittenLogFileSizeMap().keySet().stream() + .collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen)); HoodieRollbackPartitionMetadata metadata = new HoodieRollbackPartitionMetadata(stat.getPartitionPath(), - stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles()); + stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles(), rollbackLogFiles, probableLogFiles); partitionMetadataBuilder.put(stat.getPartitionPath(), metadata); totalDeleted += stat.getSuccessDeleteFiles().size(); } @@ -146,14 +153,22 @@ public static HoodieRollbackMetadata deserializeHoodieRollbackMetadata(byte[] by return deserializeAvroMetadata(bytes, HoodieRollbackMetadata.class); } + public static HoodieRestoreMetadata deserializeHoodieRestoreMetadata(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieRestoreMetadata.class); + } + public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.class); } - public static HoodieRequestedReplaceMetadata deserializeRequestedReplaceMetadta(byte[] bytes) throws IOException { + public static HoodieRequestedReplaceMetadata deserializeRequestedReplaceMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieRequestedReplaceMetadata.class); } + public static HoodieReplaceCommitMetadata deserializeHoodieReplaceMetadata(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieReplaceCommitMetadata.class); + } + public static T deserializeAvroMetadata(byte[] bytes, Class clazz) throws IOException { DatumReader reader = new SpecificDatumReader<>(clazz); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index 95a2ae618cfc3..0490e4ec5eede 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -21,14 +21,17 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import java.io.IOException; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -63,6 +66,17 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { return commitMetadata.getPartitionToWriteStats().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions written at " + s, e); + } + case HoodieTimeline.REPLACE_COMMIT_ACTION: + try { + HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( + timeline.getInstantDetails(s).get(), HoodieReplaceCommitMetadata.class); + Set partitions = new HashSet<>(); + partitions.addAll(commitMetadata.getPartitionToReplaceFileIds().keySet()); + partitions.addAll(commitMetadata.getPartitionToWriteStats().keySet()); + return partitions.stream(); + } catch (IOException e) { + throw new HoodieIOException("Failed to get partitions modified at " + s, e); } case HoodieTimeline.CLEAN_ACTION: try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 1dd6b006b7a30..3f457153d081a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -62,6 +62,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; @@ -276,7 +277,7 @@ private void ensurePartitionLoadedCorrectly(String partition) { Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr); FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath); long beginLsTs = System.currentTimeMillis(); - FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath); + FileStatus[] statuses = listPartition(partitionPath); long endLsTs = System.currentTimeMillis(); LOG.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken =" + (endLsTs - beginLsTs)); @@ -297,6 +298,16 @@ private void ensurePartitionLoadedCorrectly(String partition) { }); } + /** + * Return all the files from the partition. + * + * @param partitionPath The absolute path of the partition + * @throws IOException + */ + protected FileStatus[] listPartition(Path partitionPath) throws IOException { + return metaClient.getFs().listStatus(partitionPath); + } + /** * Helper to convert file-status to base-files. * @@ -680,6 +691,16 @@ public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitT return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxCommitTime)); } + @Override + public Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath) { + return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBefore(fg.getFileGroupId(), maxCommitTime)); + } + + @Override + public Stream getAllReplacedFileGroups(String partitionPath) { + return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplaced(fg.getFileGroupId())); + } + @Override public final Stream> getFileGroupsInPendingClustering() { try { @@ -1031,6 +1052,15 @@ private boolean isFileGroupReplacedBeforeAny(HoodieFileGroupId fileGroupId, List return isFileGroupReplacedBeforeOrOn(fileGroupId, instants.stream().max(Comparator.naturalOrder()).get()); } + private boolean isFileGroupReplacedBefore(HoodieFileGroupId fileGroupId, String instant) { + Option hoodieInstantOption = getReplaceInstant(fileGroupId); + if (!hoodieInstantOption.isPresent()) { + return false; + } + + return HoodieTimeline.compareTimestamps(instant, GREATER_THAN, hoodieInstantOption.get().getTimestamp()); + } + private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, String instant) { Option hoodieInstantOption = getReplaceInstant(fileGroupId); if (!hoodieInstantOption.isPresent()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java index d31018123c3a5..6f0e7d54717b0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java @@ -18,10 +18,16 @@ package org.apache.hudi.common.table.view; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.function.SerializableSupplier; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Functions.Function2; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.metadata.HoodieMetadataFileSystemView; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -60,9 +66,9 @@ public class FileSystemViewManager { // Factory Map to create file-system views private final Function2 viewCreator; - public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig, + private FileSystemViewManager(HoodieEngineContext context, FileSystemViewStorageConfig viewStorageConfig, Function2 viewCreator) { - this.conf = new SerializableConfiguration(conf); + this.conf = context.getHadoopConf(); this.viewStorageConfig = viewStorageConfig; this.globalViewMap = new ConcurrentHashMap<>(); this.viewCreator = viewCreator; @@ -108,8 +114,10 @@ public SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient * Closes all views opened. */ public void close() { - this.globalViewMap.values().forEach(SyncableFileSystemView::close); - this.globalViewMap.clear(); + if (!this.globalViewMap.isEmpty()) { + this.globalViewMap.values().forEach(SyncableFileSystemView::close); + this.globalViewMap.clear(); + } } // FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS @@ -145,19 +153,32 @@ private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystem /** * Create an in-memory file System view for a table. - * - * @param conf Hadoop Configuration - * @param viewConf View Storage Configuration - * @param metaClient HoodieTableMetaClient - * @return + * */ - private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf, - FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) { + private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMetadataConfig metadataConfig, FileSystemViewStorageConfig viewConf, + HoodieTableMetaClient metaClient, SerializableSupplier metadataSupplier) { LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath()); HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); + if (metadataConfig.useFileListingMetadata()) { + ValidationUtils.checkArgument(metadataSupplier != null, "Metadata supplier is null. Cannot instantiate metadata file system view"); + return new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), + metadataSupplier.get()); + } return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled()); } + public static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, + HoodieMetadataConfig metadataConfig) { + LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath()); + if (metadataConfig.useFileListingMetadata()) { + return new HoodieMetadataFileSystemView(engineContext, metaClient, + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), + metadataConfig); + } + return new HoodieTableFileSystemView(metaClient, + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); + } + /** * Create a remote file System view for a table. * @@ -175,42 +196,56 @@ private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(Serial metaClient, viewConf.getRemoteTimelineClientTimeoutSecs()); } + public static FileSystemViewManager createViewManager(final HoodieEngineContext context, + final HoodieMetadataConfig metadataConfig, + final FileSystemViewStorageConfig config) { + return createViewManager(context, metadataConfig, config, (SerializableSupplier) null); + } + + public static FileSystemViewManager createViewManager(final HoodieEngineContext context, + final HoodieMetadataConfig metadataConfig, + final FileSystemViewStorageConfig config, + final String basePath) { + return createViewManager(context, metadataConfig, config, + () -> HoodieTableMetadata.create(context, metadataConfig, basePath, config.getSpillableDir())); + } + /** * Main Factory method for building file-system views. - * - * @param conf Hadoop Configuration - * @param config View Storage Configuration - * @return + * */ - public static FileSystemViewManager createViewManager(final SerializableConfiguration conf, - final FileSystemViewStorageConfig config) { + public static FileSystemViewManager createViewManager(final HoodieEngineContext context, + final HoodieMetadataConfig metadataConfig, + final FileSystemViewStorageConfig config, + final SerializableSupplier metadataSupplier) { LOG.info("Creating View Manager with storage type :" + config.getStorageType()); + final SerializableConfiguration conf = context.getHadoopConf(); switch (config.getStorageType()) { case EMBEDDED_KV_STORE: LOG.info("Creating embedded rocks-db based Table View"); - return new FileSystemViewManager(conf, config, + return new FileSystemViewManager(context, config, (metaClient, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, metaClient)); case SPILLABLE_DISK: LOG.info("Creating Spillable Disk based Table View"); - return new FileSystemViewManager(conf, config, + return new FileSystemViewManager(context, config, (metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient)); case MEMORY: LOG.info("Creating in-memory based Table View"); - return new FileSystemViewManager(conf, config, - (metaClient, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, metaClient)); + return new FileSystemViewManager(context, config, + (metaClient, viewConfig) -> createInMemoryFileSystemView(metadataConfig, viewConfig, metaClient, metadataSupplier)); case REMOTE_ONLY: LOG.info("Creating remote only table view"); - return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf, + return new FileSystemViewManager(context, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf, viewConfig, metaClient)); case REMOTE_FIRST: LOG.info("Creating remote first table view"); - return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> { + return new FileSystemViewManager(context, config, (metaClient, viewConfig) -> { RemoteHoodieTableFileSystemView remoteFileSystemView = createRemoteFileSystemView(conf, viewConfig, metaClient); SyncableFileSystemView secondaryView; switch (viewConfig.getSecondaryStorageType()) { case MEMORY: - secondaryView = createInMemoryFileSystemView(conf, viewConfig, metaClient); + secondaryView = createInMemoryFileSystemView(metadataConfig, viewConfig, metaClient, metadataSupplier); break; case EMBEDDED_KV_STORE: secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, metaClient); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java index ff3a78f77f29d..f967d433c94b8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java @@ -33,39 +33,49 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { // Property Names public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type"; + public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; + public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable"; + public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false"; + public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type"; + public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; + public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host"; + public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost"; + public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port"; + public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754; + public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir"; + public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/"; + public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem"; + private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB + public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = "hoodie.filesystem.view.spillable.compaction.mem.fraction"; + private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01; + public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = "hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction"; + public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = "hoodie.filesystem.view.spillable.replaced.mem.fraction"; + private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01; + public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = "hoodie.filesystem.view.spillable.clustering.mem.fraction"; - private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path"; - public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = - "hoodie.filesystem.view.remote.timeout.secs"; - + private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01; - public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; - public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; + private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path"; public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb"; - public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false"; - public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost"; - public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754; + public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = + "hoodie.filesystem.view.remote.timeout.secs"; public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = 5 * 60; // 5 min - public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/"; - private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01; + private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05; - private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01; - private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01; - private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB /** * Configs to control whether backup needs to be configured if clients were not able to reach @@ -135,7 +145,7 @@ public long getMaxMemoryForPendingClusteringFileGroups() { .longValue(); } - public String getBaseStoreDir() { + public String getSpillableDir() { return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java new file mode 100644 index 0000000000000..132a9f804edf2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieMetadataException.java @@ -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. + */ + +package org.apache.hudi.exception; + +/** + *

+ * Exception thrown for table metadata related failures. + *

+ */ +public class HoodieMetadataException extends HoodieException { + public HoodieMetadataException(String msg, Exception t) { + super(msg, t); + } + + public HoodieMetadataException(String msg) { + super(msg); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index f7244eefdf9d4..3783d00b65d2b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -199,6 +199,16 @@ public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitT return execute(maxCommitTime, partitionPath, preferredView::getReplacedFileGroupsBeforeOrOn, secondaryView::getReplacedFileGroupsBeforeOrOn); } + @Override + public Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath) { + return execute(maxCommitTime, partitionPath, preferredView::getReplacedFileGroupsBefore, secondaryView::getReplacedFileGroupsBefore); + } + + @Override + public Stream getAllReplacedFileGroups(String partitionPath) { + return execute(partitionPath, preferredView::getAllReplacedFileGroups, secondaryView::getAllReplacedFileGroups); + } + @Override public Stream> getPendingCompactionOperations() { return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index d42592bd1ece4..23b0536c240dc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -91,6 +91,12 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, public static final String ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON = String.format("%s/%s", BASE_URL, "filegroups/replaced/beforeoron/"); + public static final String ALL_REPLACED_FILEGROUPS_BEFORE = + String.format("%s/%s", BASE_URL, "filegroups/replaced/before/"); + + public static final String ALL_REPLACED_FILEGROUPS_PARTITION = + String.format("%s/%s", BASE_URL, "filegroups/replaced/partition/"); + public static final String PENDING_CLUSTERING_FILEGROUPS = String.format("%s/%s", BASE_URL, "clustering/pending/"); @@ -171,7 +177,7 @@ private T executeRequest(String requestPath, Map queryParame break; } String content = response.returnContent().asString(); - return mapper.readValue(content, reference); + return (T) mapper.readValue(content, reference); } private Map getParamsWithPartitionPath(String partitionPath) { @@ -380,6 +386,30 @@ public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitT } } + @Override + public Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath) { + Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime); + try { + List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + + @Override + public Stream getAllReplacedFileGroups(String partitionPath) { + Map paramsMap = getParamsWithPartitionPath(partitionPath); + try { + List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_PARTITION, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + public boolean refresh() { Map paramsMap = getParams(); try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index d72516921ada1..1dafe33a8d973 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -62,7 +62,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile(); this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups(); this.maxMemoryForClusteringFileGroups = config.getMaxMemoryForPendingClusteringFileGroups(); - this.baseStoreDir = config.getBaseStoreDir(); + this.baseStoreDir = config.getSpillableDir(); init(metaClient, visibleActiveTimeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index 504f95a9ee089..7330286734a08 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -167,10 +167,20 @@ interface SliceView extends SliceViewWithLatestSlice { HoodieTimeline getTimeline(); /** - * Stream all the replaced file groups before maxCommitTime. + * Stream all the replaced file groups before or on maxCommitTime for given partition. */ Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath); + /** + * Stream all the replaced file groups before maxCommitTime for given partition. + */ + Stream getReplacedFileGroupsBefore(String maxCommitTime, String partitionPath); + + /** + * Stream all the replaced file groups for given partition. + */ + Stream getAllReplacedFileGroups(String partitionPath); + /** * Filegroups that are in pending clustering. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index c0c88c04ac151..70dfa2a2af1f4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -31,6 +31,7 @@ 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.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -68,21 +69,30 @@ public static Stream> getAllPendingClu .filter(Option::isPresent).map(Option::get); } - public static Option> getClusteringPlan(HoodieTableMetaClient metaClient, HoodieInstant requestedReplaceInstant) { + public static Option> getClusteringPlan(HoodieTableMetaClient metaClient, HoodieInstant pendingReplaceInstant) { try { - Option content = metaClient.getActiveTimeline().getInstantDetails(requestedReplaceInstant); + final HoodieInstant requestedInstant; + if (!pendingReplaceInstant.isRequested()) { + // inflight replacecommit files don't have clustering plan. + // This is because replacecommit inflight can have workload profile for 'insert_overwrite'. + // Get the plan from corresponding requested instant. + requestedInstant = HoodieTimeline.getReplaceCommitRequestedInstant(pendingReplaceInstant.getTimestamp()); + } else { + requestedInstant = pendingReplaceInstant; + } + Option content = metaClient.getActiveTimeline().getInstantDetails(requestedInstant); if (!content.isPresent() || content.get().length == 0) { // few operations create requested file without any content. Assume these are not clustering - LOG.warn("No content found in requested file for instant " + requestedReplaceInstant); + LOG.warn("No content found in requested file for instant " + pendingReplaceInstant); return Option.empty(); } - HoodieRequestedReplaceMetadata requestedReplaceMetadata = TimelineMetadataUtils.deserializeRequestedReplaceMetadta(content.get()); + HoodieRequestedReplaceMetadata requestedReplaceMetadata = TimelineMetadataUtils.deserializeRequestedReplaceMetadata(content.get()); if (WriteOperationType.CLUSTER.name().equals(requestedReplaceMetadata.getOperationType())) { - return Option.of(Pair.of(requestedReplaceInstant, requestedReplaceMetadata.getClusteringPlan())); + return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.getClusteringPlan())); } return Option.empty(); } catch (IOException e) { - throw new HoodieIOException("Error reading clustering plan " + requestedReplaceInstant.getTimestamp(), e); + throw new HoodieIOException("Error reading clustering plan " + pendingReplaceInstant.getTimestamp(), e); } } @@ -114,7 +124,11 @@ private static Stream> getFileGroupE new AbstractMap.SimpleEntry<>(entry.getLeft(), entry.getRight())); } - private static Stream getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) { + public static Stream getFileGroupsFromClusteringPlan(HoodieClusteringPlan clusteringPlan) { + return clusteringPlan.getInputGroups().stream().flatMap(ClusteringUtils::getFileGroupsFromClusteringGroup); + } + + public static Stream getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) { return group.getSlices().stream().map(slice -> new HoodieFileGroupId(slice.getPartitionPath(), slice.getFileId())); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ObjectSizeCalculator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ObjectSizeCalculator.java index 88587c78e25a1..f3944152faefb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ObjectSizeCalculator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ObjectSizeCalculator.java @@ -18,6 +18,14 @@ package org.apache.hudi.common.util; +import org.apache.hudi.common.util.jvm.MemoryLayoutSpecification; +import org.apache.hudi.common.util.jvm.HotSpotMemoryLayoutSpecification32bit; +import org.apache.hudi.common.util.jvm.HotSpotMemoryLayoutSpecification64bit; +import org.apache.hudi.common.util.jvm.HotSpotMemoryLayoutSpecification64bitCompressed; +import org.apache.hudi.common.util.jvm.OpenJ9MemoryLayoutSpecification32bit; +import org.apache.hudi.common.util.jvm.OpenJ9MemoryLayoutSpecification64bit; +import org.apache.hudi.common.util.jvm.OpenJ9MemoryLayoutSpecification64bitCompressed; + import java.lang.management.ManagementFactory; import java.lang.management.MemoryPoolMXBean; import java.lang.reflect.Array; @@ -35,7 +43,7 @@ import java.util.Set; /** - * Contains utility methods for calculating the memory usage of objects. It only works on the HotSpot JVM, and infers + * Contains utility methods for calculating the memory usage of objects. It only works on the HotSpot and OpenJ9 JVMs, and infers * the actual memory layout (32 bit vs. 64 bit word size, compressed object pointers vs. uncompressed) from best * available indicators. It can reliably detect a 32 bit vs. 64 bit JVM. It can only make an educated guess at whether * compressed OOPs are used, though; specifically, it knows what the JVM's default choice of OOP compression would be @@ -46,48 +54,6 @@ * @author Attila Szegedi */ public class ObjectSizeCalculator { - - /** - * Describes constant memory overheads for various constructs in a JVM implementation. - */ - public interface MemoryLayoutSpecification { - - /** - * Returns the fixed overhead of an array of any type or length in this JVM. - * - * @return the fixed overhead of an array. - */ - int getArrayHeaderSize(); - - /** - * Returns the fixed overhead of for any {@link Object} subclass in this JVM. - * - * @return the fixed overhead of any object. - */ - int getObjectHeaderSize(); - - /** - * Returns the quantum field size for a field owned by an object in this JVM. - * - * @return the quantum field size for an object. - */ - int getObjectPadding(); - - /** - * Returns the fixed size of an object reference in this JVM. - * - * @return the size of all object references. - */ - int getReferenceSize(); - - /** - * Returns the quantum field size for a field owned by one of an object's ancestor superclasses in this JVM. - * - * @return the quantum field size for a superclass field. - */ - int getSuperclassFieldPadding(); - } - private static class CurrentLayout { private static final MemoryLayoutSpecification SPEC = getEffectiveMemoryLayoutSpecification(); @@ -328,109 +294,59 @@ private static long getPrimitiveFieldSize(Class type) { static MemoryLayoutSpecification getEffectiveMemoryLayoutSpecification() { final String vmName = System.getProperty("java.vm.name"); if (vmName == null || !(vmName.startsWith("Java HotSpot(TM) ") || vmName.startsWith("OpenJDK") - || vmName.startsWith("TwitterJDK"))) { - throw new UnsupportedOperationException("ObjectSizeCalculator only supported on HotSpot VM"); - } - - final String dataModel = System.getProperty("sun.arch.data.model"); - if ("32".equals(dataModel)) { - // Running with 32-bit data model - return new MemoryLayoutSpecification() { - @Override - public int getArrayHeaderSize() { - return 12; - } - - @Override - public int getObjectHeaderSize() { - return 8; - } - - @Override - public int getObjectPadding() { - return 8; - } - - @Override - public int getReferenceSize() { - return 4; - } - - @Override - public int getSuperclassFieldPadding() { - return 4; - } - }; - } else if (!"64".equals(dataModel)) { - throw new UnsupportedOperationException( - "Unrecognized value '" + dataModel + "' of sun.arch.data.model system property"); + || vmName.startsWith("TwitterJDK") || vmName.startsWith("Eclipse OpenJ9"))) { + throw new UnsupportedOperationException("ObjectSizeCalculator only supported on HotSpot or Eclipse OpenJ9 VMs"); } final String strVmVersion = System.getProperty("java.vm.version"); - final int vmVersion = Integer.parseInt(strVmVersion.substring(0, strVmVersion.indexOf('.'))); - if (vmVersion >= 17) { + // Support for OpenJ9 JVM + if (strVmVersion.startsWith("openj9")) { + final String dataModel = System.getProperty("sun.arch.data.model"); + if ("32".equals(dataModel)) { + // Running with 32-bit data model + return new OpenJ9MemoryLayoutSpecification32bit(); + } else if (!"64".equals(dataModel)) { + throw new UnsupportedOperationException( + "Unrecognized value '" + dataModel + "' of sun.arch.data.model system property"); + } + long maxMemory = 0; for (MemoryPoolMXBean mp : ManagementFactory.getMemoryPoolMXBeans()) { maxMemory += mp.getUsage().getMax(); } - if (maxMemory < 30L * 1024 * 1024 * 1024) { - // HotSpot 17.0 and above use compressed OOPs below 30GB of RAM total - // for all memory pools (yes, including code cache). - return new MemoryLayoutSpecification() { - @Override - public int getArrayHeaderSize() { - return 16; - } - - @Override - public int getObjectHeaderSize() { - return 12; - } - - @Override - public int getObjectPadding() { - return 8; - } - - @Override - public int getReferenceSize() { - return 4; - } - - @Override - public int getSuperclassFieldPadding() { - return 4; - } - }; - } - } - - // In other cases, it's a 64-bit uncompressed OOPs object model - return new MemoryLayoutSpecification() { - @Override - public int getArrayHeaderSize() { - return 24; - } - - @Override - public int getObjectHeaderSize() { - return 16; + if (maxMemory < 57L * 1024 * 1024 * 1024) { + // OpenJ9 use compressed references below 57GB of RAM total + return new OpenJ9MemoryLayoutSpecification64bitCompressed(); + } else { + // it's a 64-bit uncompressed references object model + return new OpenJ9MemoryLayoutSpecification64bit(); } - - @Override - public int getObjectPadding() { - return 8; + } else { + // Support for HotSpot JVM + final String dataModel = System.getProperty("sun.arch.data.model"); + if ("32".equals(dataModel)) { + // Running with 32-bit data model + return new HotSpotMemoryLayoutSpecification32bit(); + } else if (!"64".equals(dataModel)) { + throw new UnsupportedOperationException( + "Unrecognized value '" + dataModel + "' of sun.arch.data.model system property"); } - @Override - public int getReferenceSize() { - return 8; + final int vmVersion = Integer.parseInt(strVmVersion.substring(0, strVmVersion.indexOf('.'))); + if (vmVersion >= 17) { + long maxMemory = 0; + for (MemoryPoolMXBean mp : ManagementFactory.getMemoryPoolMXBeans()) { + maxMemory += mp.getUsage().getMax(); + } + if (maxMemory < 30L * 1024 * 1024 * 1024) { + // HotSpot 17.0 and above use compressed OOPs below 30GB of RAM total + // for all memory pools (yes, including code cache). + return new HotSpotMemoryLayoutSpecification64bitCompressed(); + } } - @Override - public int getSuperclassFieldPadding() { - return 8; - } - }; + // In other cases, it's a 64-bit uncompressed OOPs object model + return new HotSpotMemoryLayoutSpecification64bit(); + } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index eb5e2b5fa068f..dc444aa21bf14 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -39,6 +39,7 @@ import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; @@ -261,6 +262,22 @@ public static List readAvroRecords(Configuration configuration, P return records; } + /** + * Returns the number of records in the parquet file. + * + * @param conf Configuration + * @param parquetFilePath path of the file + */ + public static long getRowCount(Configuration conf, Path parquetFilePath) { + ParquetMetadata footer; + long rowCount = 0; + footer = readMetadata(conf, parquetFilePath); + for (BlockMetaData b : footer.getBlocks()) { + rowCount += b.getRowCount(); + } + return rowCount; + } + static class RecordKeysFilterFunction implements Function { private final Set candidateKeys; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index 49f1075508c86..326bf05277f0b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -25,6 +25,8 @@ */ public class StringUtils { + public static final String EMPTY_STRING = ""; + /** *

* Joins the elements of the provided array into a single String containing the provided list of elements. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification32bit.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification32bit.java new file mode 100644 index 0000000000000..e76c67e39276b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification32bit.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class HotSpotMemoryLayoutSpecification32bit implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 12; + } + + @Override + public int getObjectHeaderSize() { + return 8; + } + + @Override + public int getObjectPadding() { + return 8; + } + + @Override + public int getReferenceSize() { + return 4; + } + + @Override + public int getSuperclassFieldPadding() { + return 4; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bit.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bit.java new file mode 100644 index 0000000000000..4f4e2a4b6cabc --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bit.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class HotSpotMemoryLayoutSpecification64bit implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 24; + } + + @Override + public int getObjectHeaderSize() { + return 16; + } + + @Override + public int getObjectPadding() { + return 8; + } + + @Override + public int getReferenceSize() { + return 8; + } + + @Override + public int getSuperclassFieldPadding() { + return 8; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bitCompressed.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bitCompressed.java new file mode 100644 index 0000000000000..60ad8be288991 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/HotSpotMemoryLayoutSpecification64bitCompressed.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class HotSpotMemoryLayoutSpecification64bitCompressed implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 16; + } + + @Override + public int getObjectHeaderSize() { + return 12; + } + + @Override + public int getObjectPadding() { + return 8; + } + + @Override + public int getReferenceSize() { + return 4; + } + + @Override + public int getSuperclassFieldPadding() { + return 4; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/MemoryLayoutSpecification.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/MemoryLayoutSpecification.java new file mode 100644 index 0000000000000..fe6d421e5c357 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/MemoryLayoutSpecification.java @@ -0,0 +1,60 @@ +/* + * 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.util.jvm; + +/** + * Describes constant memory overheads for various constructs in a JVM implementation. + */ +public interface MemoryLayoutSpecification { + + /** + * Returns the fixed overhead of an array of any type or length in this JVM. + * + * @return the fixed overhead of an array. + */ + int getArrayHeaderSize(); + + /** + * Returns the fixed overhead of for any {@link Object} subclass in this JVM. + * + * @return the fixed overhead of any object. + */ + int getObjectHeaderSize(); + + /** + * Returns the quantum field size for a field owned by an object in this JVM. + * + * @return the quantum field size for an object. + */ + int getObjectPadding(); + + /** + * Returns the fixed size of an object reference in this JVM. + * + * @return the size of all object references. + */ + int getReferenceSize(); + + /** + * Returns the quantum field size for a field owned by one of an object's ancestor superclasses in this JVM. + * + * @return the quantum field size for a superclass field. + */ + int getSuperclassFieldPadding(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification32bit.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification32bit.java new file mode 100644 index 0000000000000..1e66e0cecf122 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification32bit.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class OpenJ9MemoryLayoutSpecification32bit implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 16; + } + + @Override + public int getObjectHeaderSize() { + return 4; + } + + @Override + public int getObjectPadding() { + return 4; + } + + @Override + public int getReferenceSize() { + return 4; + } + + @Override + public int getSuperclassFieldPadding() { + return 4; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bit.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bit.java new file mode 100644 index 0000000000000..d89d05ae475b2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bit.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class OpenJ9MemoryLayoutSpecification64bit implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 16; + } + + @Override + public int getObjectHeaderSize() { + return 16; + } + + @Override + public int getObjectPadding() { + return 8; + } + + @Override + public int getReferenceSize() { + return 8; + } + + @Override + public int getSuperclassFieldPadding() { + return 8; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bitCompressed.java b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bitCompressed.java new file mode 100644 index 0000000000000..05a457525a361 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/jvm/OpenJ9MemoryLayoutSpecification64bitCompressed.java @@ -0,0 +1,46 @@ +/* + * 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.util.jvm; + +public class OpenJ9MemoryLayoutSpecification64bitCompressed implements MemoryLayoutSpecification { + @Override + public int getArrayHeaderSize() { + return 16; + } + + @Override + public int getObjectHeaderSize() { + return 4; + } + + @Override + public int getObjectPadding() { + return 4; + } + + @Override + public int getReferenceSize() { + return 4; + } + + @Override + public int getSuperclassFieldPadding() { + return 4; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java b/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java index 7666e90a74f90..2febe51f7710c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java +++ b/hudi-common/src/main/java/org/apache/hudi/exception/TableNotFoundException.java @@ -18,9 +18,11 @@ package org.apache.hudi.exception; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import java.io.FileNotFoundException; import java.io.IOException; /** @@ -39,16 +41,14 @@ private static String getErrorMessage(String basePath) { } public static void checkTableValidity(FileSystem fs, Path basePathDir, Path metaPathDir) { - // Check if the base path is found + // Check if the base and meta paths are found try { - if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) { - throw new TableNotFoundException(basePathDir.toString()); - } - // Check if the meta path is found - if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) { + // Since metaPath is within the basePath, it is enough to check the metaPath exists + FileStatus status = fs.getFileStatus(metaPathDir); + if (!status.isDirectory()) { throw new TableNotFoundException(metaPathDir.toString()); } - } catch (IllegalArgumentException e) { + } catch (FileNotFoundException | IllegalArgumentException e) { // if the base path is file:///, then we have a IllegalArgumentException throw new TableNotFoundException(metaPathDir.toString()); } catch (IOException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java index 107f50318eba9..feacbda54606b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetReader.java @@ -74,7 +74,6 @@ public void close() { @Override public long getTotalRecords() { - // TODO Auto-generated method stub - return 0; + return ParquetUtils.getRowCount(conf, path); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java new file mode 100644 index 0000000000000..c86b37e09799c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -0,0 +1,308 @@ + +/* + * 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.metadata; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieMetadataException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public abstract class BaseTableMetadata implements HoodieTableMetadata { + + private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class); + + static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; + static final int BUFFER_SIZE = 10 * 1024 * 1024; + + protected final transient HoodieEngineContext engineContext; + protected final SerializableConfiguration hadoopConf; + protected final String datasetBasePath; + protected final HoodieTableMetaClient datasetMetaClient; + protected final Option metrics; + protected final HoodieMetadataConfig metadataConfig; + // Directory used for Spillable Map when merging records + protected final String spillableMapDirectory; + + protected boolean enabled; + private TimelineMergedTableMetadata timelineMergedMetadata; + + protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, + String datasetBasePath, String spillableMapDirectory) { + this.engineContext = engineContext; + this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf()); + this.datasetBasePath = datasetBasePath; + this.datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath); + this.spillableMapDirectory = spillableMapDirectory; + this.metadataConfig = metadataConfig; + + this.enabled = metadataConfig.useFileListingMetadata(); + if (metadataConfig.enableMetrics()) { + this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata"))); + } else { + this.metrics = Option.empty(); + } + if (enabled) { + openTimelineScanner(); + } + } + + /** + * Return the list of partitions in the dataset. + * + * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of + * partitions is retrieved directly from the underlying {@code FileSystem}. + * + * On any errors retrieving the listing from the metadata, defaults to using the file system listings. + * + */ + @Override + public List getAllPartitionPaths() throws IOException { + if (enabled) { + try { + return fetchAllPartitionPaths(); + } catch (Exception e) { + if (metadataConfig.enableFallback()) { + LOG.error("Failed to retrieve list of partition from metadata", e); + } else { + throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); + } + } + } + return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, + metadataConfig.shouldAssumeDatePartitioning()).getAllPartitionPaths(); + } + + /** + * Return the list of files in a partition. + * + * If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of + * partitions is retrieved directly from the underlying {@code FileSystem}. + * + * On any errors retrieving the listing from the metadata, defaults to using the file system listings. + * + * @param partitionPath The absolute path of the partition to list + */ + @Override + public FileStatus[] getAllFilesInPartition(Path partitionPath) + throws IOException { + if (enabled) { + try { + return fetchAllFilesInPartition(partitionPath); + } catch (Exception e) { + if (metadataConfig.enableFallback()) { + LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e); + } else { + throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e); + } + } + } + + return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()) + .getAllFilesInPartition(partitionPath); + } + + /** + * Returns a list of all partitions. + */ + protected List fetchAllPartitionPaths() throws IOException { + HoodieTimer timer = new HoodieTimer().startTimer(); + Option> hoodieRecord = getMergedRecordByKey(RECORDKEY_PARTITION_LIST); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer())); + + List partitions = Collections.emptyList(); + if (hoodieRecord.isPresent()) { + if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { + throw new HoodieMetadataException("Metadata partition list record is inconsistent: " + + hoodieRecord.get().getData()); + } + + partitions = hoodieRecord.get().getData().getFilenames(); + // Partition-less tables have a single empty partition + if (partitions.contains(NON_PARTITIONED_NAME)) { + partitions.remove(NON_PARTITIONED_NAME); + partitions.add(""); + } + } + + if (metadataConfig.validateFileListingMetadata()) { + // Validate the Metadata Table data by listing the partitions from the file system + timer.startTimer(); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(getEngineContext(), + hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()); + List actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths(); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_PARTITIONS_STR, timer.endTimer())); + + Collections.sort(actualPartitions); + Collections.sort(partitions); + if (!actualPartitions.equals(partitions)) { + LOG.error("Validation of metadata partition list failed. Lists do not match."); + LOG.error("Partitions from metadata: " + Arrays.toString(partitions.toArray())); + LOG.error("Partitions from file system: " + Arrays.toString(actualPartitions.toArray())); + + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); + } + + // Return the direct listing as it should be correct + partitions = actualPartitions; + } + + LOG.info("Listed partitions from metadata: #partitions=" + partitions.size()); + return partitions; + } + + /** + * Return all the files from the partition. + * + * @param partitionPath The absolute path of the partition + */ + FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { + String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), partitionPath); + if (partitionName.isEmpty()) { + partitionName = NON_PARTITIONED_NAME; + } + + HoodieTimer timer = new HoodieTimer().startTimer(); + Option> hoodieRecord = getMergedRecordByKey(partitionName); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); + + FileStatus[] statuses = {}; + if (hoodieRecord.isPresent()) { + if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { + throw new HoodieMetadataException("Metadata record for partition " + partitionName + " is inconsistent: " + + hoodieRecord.get().getData()); + } + statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath); + } + + if (metadataConfig.validateFileListingMetadata()) { + // Validate the Metadata Table data by listing the partitions from the file system + timer.startTimer(); + + String partitionPathStr = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), partitionPath); + String latestDataInstantTime = getLatestDatasetInstantTime(); + HoodieTableFileSystemView dataFsView = new HoodieTableFileSystemView(datasetMetaClient, datasetMetaClient.getActiveTimeline()); + List directStatuses = dataFsView.getAllFileSlices(partitionPathStr).flatMap(slice -> { + List paths = new ArrayList<>(); + slice.getBaseFile().ifPresent(baseFile -> { + if (HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, latestDataInstantTime)) { + paths.add(baseFile.getFileStatus()); + } + }); + //TODO(metadata): this will remain problematic; no way to know the commit time based on log file written + slice.getLogFiles().forEach(logFile -> paths.add(logFile.getFileStatus())); + return paths.stream(); + }).collect(Collectors.toList()); + + List directFilenames = directStatuses.stream() + .map(fileStatus -> fileStatus.getPath().getName()).sorted() + .collect(Collectors.toList()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer())); + + List metadataFilenames = Arrays.stream(statuses) + .map(s -> s.getPath().getName()).sorted() + .collect(Collectors.toList()); + + if (!metadataFilenames.equals(directFilenames)) { + LOG.error("Validation of metadata file listing for partition " + partitionName + " failed."); + LOG.error("File list from metadata: " + Arrays.toString(metadataFilenames.toArray())); + LOG.error("File list from direct listing: " + Arrays.toString(directFilenames.toArray())); + + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0)); + } + + // Return the direct listing as it should be correct + statuses = directStatuses.toArray(new FileStatus[0]); + } + + LOG.info("Listed file in partition from metadata: partition=" + partitionName + ", #files=" + statuses.length); + return statuses; + } + + /** + * Retrieve the merged {@code HoodieRecord} mapped to the given key. + * + * @param key The key of the record + */ + private Option> getMergedRecordByKey(String key) { + Option> mergedRecord; + Option> metadataHoodieRecord = getRecordByKeyFromMetadata(key); + // Retrieve record from unsynced timeline instants + Option> timelineHoodieRecord = timelineMergedMetadata.getRecordByKey(key); + if (timelineHoodieRecord.isPresent()) { + if (metadataHoodieRecord.isPresent()) { + HoodieRecordPayload mergedPayload = timelineHoodieRecord.get().getData().preCombine(metadataHoodieRecord.get().getData()); + mergedRecord = Option.of(new HoodieRecord(metadataHoodieRecord.get().getKey(), mergedPayload)); + } else { + mergedRecord = timelineHoodieRecord; + } + } else { + mergedRecord = metadataHoodieRecord; + } + return mergedRecord; + } + + protected abstract Option> getRecordByKeyFromMetadata(String key); + + private void openTimelineScanner() { + if (timelineMergedMetadata == null) { + List unSyncedInstants = findInstantsToSync(); + timelineMergedMetadata = + new TimelineMergedTableMetadata(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), null); + } + } + + protected abstract List findInstantsToSync(); + + public boolean isInSync() { + return enabled && findInstantsToSync().isEmpty(); + } + + protected HoodieEngineContext getEngineContext() { + return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get()); + } + + protected String getLatestDatasetInstantTime() { + return datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() + .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java new file mode 100644 index 0000000000000..31c74a1f68fd2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -0,0 +1,122 @@ +/* + * 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.metadata; + +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.stream.Collectors; + +public class FileSystemBackedTableMetadata implements HoodieTableMetadata { + + private static final int DEFAULT_LISTING_PARALLELISM = 1500; + + private final transient HoodieEngineContext engineContext; + private final SerializableConfiguration hadoopConf; + private final String datasetBasePath; + private final boolean assumeDatePartitioning; + + public FileSystemBackedTableMetadata(HoodieEngineContext engineContext, SerializableConfiguration conf, String datasetBasePath, + boolean assumeDatePartitioning) { + this.engineContext = engineContext; + this.hadoopConf = conf; + this.datasetBasePath = datasetBasePath; + this.assumeDatePartitioning = assumeDatePartitioning; + } + + @Override + public FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException { + FileSystem fs = partitionPath.getFileSystem(hadoopConf.get()); + return FSUtils.getAllDataFilesInPartition(fs, partitionPath); + } + + @Override + public List getAllPartitionPaths() throws IOException { + if (assumeDatePartitioning) { + FileSystem fs = new Path(datasetBasePath).getFileSystem(hadoopConf.get()); + return FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, datasetBasePath); + } + + List pathsToList = new LinkedList<>(); + pathsToList.add(new Path(datasetBasePath)); + List partitionPaths = new ArrayList<>(); + + while (!pathsToList.isEmpty()) { + // TODO: Get the parallelism from HoodieWriteConfig + int listingParallelism = Math.min(DEFAULT_LISTING_PARALLELISM, pathsToList.size()); + + // List all directories in parallel + List> dirToFileListing = engineContext.map(pathsToList, path -> { + FileSystem fileSystem = path.getFileSystem(hadoopConf.get()); + return Pair.of(path, fileSystem.listStatus(path)); + }, listingParallelism); + pathsToList.clear(); + + // If the listing reveals a directory, add it to queue. If the listing reveals a hoodie partition, add it to + // the results. + dirToFileListing.forEach(p -> { + Option partitionMetaFile = Option.fromJavaOptional(Arrays.stream(p.getRight()).parallel() + .filter(fs -> fs.getPath().getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) + .findFirst()); + + if (partitionMetaFile.isPresent()) { + // Is a partition. + String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), p.getLeft()); + partitionPaths.add(partitionName); + } else { + // Add sub-dirs to the queue + pathsToList.addAll(Arrays.stream(p.getRight()) + .filter(fs -> fs.isDirectory() && !fs.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) + .map(fs -> fs.getPath()) + .collect(Collectors.toList())); + } + }); + } + return partitionPaths; + } + + @Override + public Option getSyncedInstantTime() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isInSync() { + return true; + } + + @Override + public void close() throws Exception { + // no-op + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java new file mode 100644 index 0000000000000..a34652c688359 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -0,0 +1,279 @@ +/* + * 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.metadata; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Table metadata provided by an internal DFS backed Hudi metadata table. + * + * If the metadata table does not exist, RPC calls are used to retrieve file listings from the file system. + * No updates are applied to the table and it is not synced. + */ +public class HoodieBackedTableMetadata extends BaseTableMetadata { + + private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class); + + private String metadataBasePath; + // Metadata table's timeline and metaclient + private HoodieTableMetaClient metaClient; + private List latestFileSystemMetadataSlices; + + // Readers for the base and log file which store the metadata + private transient HoodieFileReader baseFileReader; + private transient HoodieMetadataMergedLogRecordScanner logRecordScanner; + + public HoodieBackedTableMetadata(Configuration conf, HoodieMetadataConfig metadataConfig, + String datasetBasePath, String spillableMapDirectory) { + this(new HoodieLocalEngineContext(conf), metadataConfig, datasetBasePath, spillableMapDirectory); + } + + public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, + String datasetBasePath, String spillableMapDirectory) { + super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory); + initIfNeeded(); + } + + private void initIfNeeded() { + if (enabled && this.metaClient == null) { + this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath); + try { + this.metaClient = new HoodieTableMetaClient(hadoopConf.get(), metadataBasePath); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + latestFileSystemMetadataSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + } catch (TableNotFoundException e) { + LOG.warn("Metadata table was not found at path " + metadataBasePath); + this.enabled = false; + this.metaClient = null; + } catch (Exception e) { + LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e); + this.enabled = false; + this.metaClient = null; + } + } else { + LOG.info("Metadata table is disabled."); + } + } + + @Override + protected Option> getRecordByKeyFromMetadata(String key) { + try { + List timings = new ArrayList<>(); + HoodieTimer timer = new HoodieTimer().startTimer(); + openFileSliceIfNeeded(); + timings.add(timer.endTimer()); + + timer.startTimer(); + // Retrieve record from base file + HoodieRecord hoodieRecord = null; + if (baseFileReader != null) { + HoodieTimer readTimer = new HoodieTimer().startTimer(); + Option baseRecord = baseFileReader.getRecordByKey(key); + if (baseRecord.isPresent()) { + hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), + metaClient.getTableConfig().getPayloadClass()); + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer())); + } + } + timings.add(timer.endTimer()); + + // Retrieve record from log file + timer.startTimer(); + if (logRecordScanner != null) { + Option> logHoodieRecord = logRecordScanner.getRecordByKey(key); + if (logHoodieRecord.isPresent()) { + if (hoodieRecord != null) { + // Merge the payloads + HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData()); + hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload); + } else { + hoodieRecord = logHoodieRecord.get(); + } + } + } + timings.add(timer.endTimer()); + LOG.info(String.format("Metadata read for key %s took [open, baseFileRead, logMerge] %s ms", key, timings)); + return Option.ofNullable(hoodieRecord); + } catch (IOException ioe) { + throw new HoodieIOException("Error merging records from metadata table for key :" + key, ioe); + } finally { + closeIfNeeded(); + } + } + + /** + * Open readers to the base and log files. + */ + private synchronized void openFileSliceIfNeeded() throws IOException { + if (metadataConfig.enableReuse() && baseFileReader != null) { + // we will reuse what's open. + return; + } + + // Metadata is in sync till the latest completed instant on the dataset + HoodieTimer timer = new HoodieTimer().startTimer(); + String latestInstantTime = getLatestDatasetInstantTime(); + ValidationUtils.checkArgument(latestFileSystemMetadataSlices.size() == 1, "must be at-least one validata metadata file slice"); + + // If the base file is present then create a reader + Option basefile = latestFileSystemMetadataSlices.get(0).getBaseFile(); + if (basefile.isPresent()) { + String basefilePath = basefile.get().getPath(); + baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath)); + LOG.info("Opened metadata base file from " + basefilePath + " at instant " + basefile.get().getCommitTime()); + } + + // Open the log record scanner using the log files from the latest file slice + List logFilePaths = latestFileSystemMetadataSlices.get(0).getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(o -> o.getPath().toString()) + .collect(Collectors.toList()); + Option lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); + String latestMetaInstantTimestamp = lastInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + + // Load the schema + Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema()); + logRecordScanner = new HoodieMetadataMergedLogRecordScanner(metaClient.getFs(), metadataBasePath, + logFilePaths, schema, latestMetaInstantTimestamp, MAX_MEMORY_SIZE_IN_BYTES, BUFFER_SIZE, + spillableMapDirectory, null); + + LOG.info("Opened metadata log files from " + logFilePaths + " at instant " + latestInstantTime + + "(dataset instant=" + latestInstantTime + ", metadata instant=" + latestMetaInstantTimestamp + ")"); + + metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, timer.endTimer())); + } + + private void closeIfNeeded() { + try { + if (!metadataConfig.enableReuse()) { + close(); + } + } catch (Exception e) { + throw new HoodieException("Error closing resources during metadata table merge", e); + } + } + + @Override + public void close() throws Exception { + if (baseFileReader != null) { + baseFileReader.close(); + baseFileReader = null; + } + if (logRecordScanner != null) { + logRecordScanner.close(); + logRecordScanner = null; + } + } + + /** + * Return an ordered list of instants which have not been synced to the Metadata Table. + */ + protected List findInstantsToSync() { + initIfNeeded(); + + // if there are no instants yet, return empty list, since there is nothing to sync here. + if (!enabled || !metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) { + return Collections.EMPTY_LIST; + } + + // All instants on the data timeline, which are greater than the last instant on metadata timeline + // are candidates for sync. + String latestMetadataInstantTime = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp(); + HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE); + Option earliestIncompleteInstant = candidateTimeline.filterInflightsAndRequested().firstInstant(); + + if (earliestIncompleteInstant.isPresent()) { + return candidateTimeline.filterCompletedInstants() + .findInstantsBefore(earliestIncompleteInstant.get().getTimestamp()) + .getInstants().collect(Collectors.toList()); + } else { + return candidateTimeline.filterCompletedInstants() + .getInstants().collect(Collectors.toList()); + } + } + + /** + * Return the timestamp of the latest compaction instant. + */ + @Override + public Option getSyncedInstantTime() { + if (!enabled) { + return Option.empty(); + } + + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + return timeline.getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant().map(HoodieInstant::getTimestamp); + } + + public boolean enabled() { + return enabled; + } + + public SerializableConfiguration getHadoopConf() { + return hadoopConf; + } + + public HoodieTableMetaClient getMetaClient() { + return metaClient; + } + + public Map stats() { + return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java new file mode 100644 index 0000000000000..36c9cefcfd33e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java @@ -0,0 +1,76 @@ +/* + * 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.metadata; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.HoodieException; + +/** + * {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table. + */ +public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView { + + private final HoodieTableMetadata tableMetadata; + + public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline, + HoodieTableMetadata tableMetadata) { + super(metaClient, visibleActiveTimeline); + this.tableMetadata = tableMetadata; + } + + public HoodieMetadataFileSystemView(HoodieEngineContext engineContext, + HoodieTableMetaClient metaClient, + HoodieTimeline visibleActiveTimeline, + HoodieMetadataConfig metadataConfig) { + super(metaClient, visibleActiveTimeline); + this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(), + FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR); + } + + /** + * Return all the files in the partition by reading from the Metadata Table. + * + * @param partitionPath The absolute path of the partition + * @throws IOException + */ + @Override + protected FileStatus[] listPartition(Path partitionPath) throws IOException { + return tableMetadata.getAllFilesInPartition(partitionPath); + } + + @Override + public void close() { + try { + tableMetadata.close(); + } catch (Exception e) { + throw new HoodieException("Error closing metadata file system view.", e); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java new file mode 100644 index 0000000000000..ae471dc52e799 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordScanner.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +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.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.util.Option; + +/** + * A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is + * useful in limiting memory usage when only a small subset of updates records are to be read. + */ +public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordScanner { + // Set of all record keys that are to be read in memory + private Set mergeKeyFilter; + + public HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, + Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize, + String spillableMapBasePath, Set mergeKeyFilter) { + super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize, + spillableMapBasePath, false); + this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); + + performScan(); + } + + @Override + protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException { + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieRecord.getRecordKey())) { + super.processNextRecord(hoodieRecord); + } + } + + @Override + protected void processNextDeletedKey(HoodieKey hoodieKey) { + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieKey.getRecordKey())) { + super.processNextDeletedKey(hoodieKey); + } + } + + /** + * Retrieve a record given its key. + * + * @param key Key of the record to retrieve + * @return {@code HoodieRecord} if key was found else {@code Option.empty()} + */ + public Option> getRecordByKey(String key) { + return Option.ofNullable((HoodieRecord) records.get(key)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java new file mode 100644 index 0000000000000..2bd773bc7819f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMetrics.java @@ -0,0 +1,147 @@ +/* + * 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.metadata; + +import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HoodieMetadataMetrics implements Serializable { + + // Metric names + public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions"; + public static final String LOOKUP_FILES_STR = "lookup_files"; + public static final String VALIDATE_PARTITIONS_STR = "validate_partitions"; + public static final String VALIDATE_FILES_STR = "validate_files"; + public static final String VALIDATE_ERRORS_STR = "validate_errors"; + public static final String SCAN_STR = "scan"; + public static final String BASEFILE_READ_STR = "basefile_read"; + public static final String INITIALIZE_STR = "initialize"; + public static final String SYNC_STR = "sync"; + + // Stats names + public static final String STAT_TOTAL_BASE_FILE_SIZE = "totalBaseFileSizeInBytes"; + public static final String STAT_TOTAL_LOG_FILE_SIZE = "totalLogFileSizeInBytes"; + public static final String STAT_COUNT_BASE_FILES = "baseFileCount"; + public static final String STAT_COUNT_LOG_FILES = "logFileCount"; + public static final String STAT_COUNT_PARTITION = "partitionCount"; + public static final String STAT_IN_SYNC = "isInSync"; + public static final String STAT_LAST_COMPACTION_TIMESTAMP = "lastCompactionTimestamp"; + + private static final Logger LOG = LogManager.getLogger(HoodieMetadataMetrics.class); + + private final Registry metricsRegistry; + + public HoodieMetadataMetrics(Registry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + } + + public Map getStats(boolean detailed, HoodieTableMetaClient metaClient, HoodieTableMetadata metadata) { + try { + metaClient.reloadActiveTimeline(); + HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); + return getStats(fsView, detailed, metadata); + } catch (IOException ioe) { + throw new HoodieIOException("Unable to get metadata stats.", ioe); + } + } + + private Map getStats(HoodieTableFileSystemView fsView, boolean detailed, HoodieTableMetadata tableMetadata) throws IOException { + Map stats = new HashMap<>(); + + // Total size of the metadata and count of base/log files + long totalBaseFileSizeInBytes = 0; + long totalLogFileSizeInBytes = 0; + int baseFileCount = 0; + int logFileCount = 0; + List latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList()); + + for (FileSlice slice : latestSlices) { + if (slice.getBaseFile().isPresent()) { + totalBaseFileSizeInBytes += slice.getBaseFile().get().getFileStatus().getLen(); + ++baseFileCount; + } + Iterator it = slice.getLogFiles().iterator(); + while (it.hasNext()) { + totalLogFileSizeInBytes += it.next().getFileStatus().getLen(); + ++logFileCount; + } + } + + stats.put(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes)); + stats.put(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes)); + stats.put(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount)); + stats.put(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES, String.valueOf(logFileCount)); + + if (detailed) { + stats.put(HoodieMetadataMetrics.STAT_COUNT_PARTITION, String.valueOf(tableMetadata.getAllPartitionPaths().size())); + stats.put(HoodieMetadataMetrics.STAT_IN_SYNC, String.valueOf(tableMetadata.isInSync())); + } + + return stats; + } + + protected void updateMetrics(String action, long durationInMs) { + if (metricsRegistry == null) { + return; + } + + // Update sum of duration and total for count + String countKey = action + ".count"; + String durationKey = action + ".totalDuration"; + metricsRegistry.add(countKey, 1); + metricsRegistry.add(durationKey, durationInMs); + + LOG.info(String.format("Updating metadata metrics (%s=%dms, %s=1)", durationKey, durationInMs, countKey)); + } + + public void updateMetrics(long totalBaseFileSizeInBytes, long totalLogFileSizeInBytes, int baseFileCount, + int logFileCount) { + if (metricsRegistry == null) { + return; + } + + // Update sizes and count for metadata table's data files + metricsRegistry.add("basefile.size", totalBaseFileSizeInBytes); + metricsRegistry.add("logfile.size", totalLogFileSizeInBytes); + metricsRegistry.add("basefile.count", baseFileCount); + metricsRegistry.add("logfile.count", logFileCount); + + LOG.info(String.format("Updating metadata size metrics (basefile.size=%d, logfile.size=%d, basefile.count=%d, " + + "logfile.count=%d)", totalBaseFileSizeInBytes, totalLogFileSizeInBytes, baseFileCount, logFileCount)); + } + + public Registry registry() { + return metricsRegistry; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java new file mode 100644 index 0000000000000..9c6eb89b909f8 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -0,0 +1,237 @@ +/* + * 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.metadata; + +import org.apache.hudi.avro.model.HoodieMetadataFileInfo; +import org.apache.hudi.avro.model.HoodieMetadataRecord; +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.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST; + +/** + * This is a payload which saves information about a single entry in the Metadata Table. + * + * The type of the entry is determined by the "type" saved within the record. The following types of entries are saved: + * + * 1. List of partitions: There is a single such record + * key="__all_partitions__" + * + * 2. List of files in a Partition: There is one such record for each partition + * key=Partition name + * + * During compaction on the table, the deletions are merged with additions and hence pruned. + * + * Metadata Table records are saved with the schema defined in HoodieMetadata.avsc. This class encapsulates the + * HoodieMetadataRecord for ease of operations. + */ +public class HoodieMetadataPayload implements HoodieRecordPayload { + // Type of the record + // This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810 + private static final int PARTITION_LIST = 1; + private static final int FILE_LIST = 2; + + private String key = null; + private int type = 0; + private Map filesystemMetadata = null; + + public HoodieMetadataPayload(Option record) { + if (record.isPresent()) { + // This can be simplified using SpecificData.deepcopy once this bug is fixed + // https://issues.apache.org/jira/browse/AVRO-1811 + key = record.get().get("key").toString(); + type = (int) record.get().get("type"); + if (record.get().get("filesystemMetadata") != null) { + filesystemMetadata = (Map) record.get().get("filesystemMetadata"); + filesystemMetadata.keySet().forEach(k -> { + GenericRecord v = filesystemMetadata.get(k); + filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted"))); + }); + } + } + } + + private HoodieMetadataPayload(String key, int type, Map filesystemMetadata) { + this.key = key; + this.type = type; + this.filesystemMetadata = filesystemMetadata; + } + + /** + * Create and return a {@code HoodieMetadataPayload} to save list of partitions. + * + * @param partitions The list of partitions + */ + public static HoodieRecord createPartitionListRecord(List partitions) { + Map fileInfo = new HashMap<>(); + partitions.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false))); + + HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), PARTITION_LIST, fileInfo); + return new HoodieRecord<>(key, payload); + } + + /** + * Create and return a {@code HoodieMetadataPayload} to save list of files within a partition. + * + * @param partition The name of the partition + * @param filesAdded Mapping of files to their sizes for files which have been added to this partition + * @param filesDeleted List of files which have been deleted from this partition + */ + public static HoodieRecord createPartitionFilesRecord(String partition, + Option> filesAdded, Option> filesDeleted) { + Map fileInfo = new HashMap<>(); + filesAdded.ifPresent( + m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false)))); + filesDeleted.ifPresent( + m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true)))); + + HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.partitionPath()); + HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), FILE_LIST, fileInfo); + return new HoodieRecord<>(key, payload); + } + + @Override + public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { + ValidationUtils.checkArgument(previousRecord.type == type, + "Cannot combine " + previousRecord.type + " with " + type); + + Map combinedFileInfo = null; + + switch (type) { + case PARTITION_LIST: + case FILE_LIST: + combinedFileInfo = combineFilesystemMetadata(previousRecord); + break; + default: + throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type); + } + + return new HoodieMetadataPayload(key, type, combinedFileInfo); + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException { + HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord)oldRecord)); + HoodieRecordPayload combinedPayload = preCombine(anotherPayload); + return combinedPayload.getInsertValue(schema); + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + if (key == null) { + return Option.empty(); + } + + HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata); + return Option.of(record); + } + + /** + * Returns the list of filenames added as part of this record. + */ + public List getFilenames() { + return filterFileInfoEntries(false).map(e -> e.getKey()).sorted().collect(Collectors.toList()); + } + + /** + * Returns the list of filenames deleted as part of this record. + */ + public List getDeletions() { + return filterFileInfoEntries(true).map(Map.Entry::getKey).sorted().collect(Collectors.toList()); + } + + /** + * Returns the files added as part of this record. + */ + public FileStatus[] getFileStatuses(Configuration hadoopConf, Path partitionPath) throws IOException { + FileSystem fs = partitionPath.getFileSystem(hadoopConf); + long blockSize = fs.getDefaultBlockSize(partitionPath); + return filterFileInfoEntries(false) + .map(e -> new FileStatus(e.getValue().getSize(), false, 0, blockSize, 0, 0, + null, null, null, new Path(partitionPath, e.getKey()))) + .toArray(FileStatus[]::new); + } + + private Stream> filterFileInfoEntries(boolean isDeleted) { + if (filesystemMetadata == null) { + return Stream.empty(); + } + + return filesystemMetadata.entrySet().stream().filter(e -> e.getValue().getIsDeleted() == isDeleted); + } + + private Map combineFilesystemMetadata(HoodieMetadataPayload previousRecord) { + Map combinedFileInfo = new HashMap<>(); + if (previousRecord.filesystemMetadata != null) { + combinedFileInfo.putAll(previousRecord.filesystemMetadata); + } + + if (filesystemMetadata != null) { + filesystemMetadata.forEach((filename, fileInfo) -> { + // If the filename wasnt present then we carry it forward + if (!combinedFileInfo.containsKey(filename)) { + combinedFileInfo.put(filename, fileInfo); + } else { + if (fileInfo.getIsDeleted()) { + // file deletion + combinedFileInfo.remove(filename); + } else { + // file appends. + combinedFileInfo.merge(filename, fileInfo, (oldFileInfo, newFileInfo) -> { + return new HoodieMetadataFileInfo(oldFileInfo.getSize() + newFileInfo.getSize(), false); + }); + } + } + }); + } + + return combinedFileInfo; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {"); + sb.append("key=").append(key).append(", "); + sb.append("type=").append(type).append(", "); + sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", "); + sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", "); + sb.append('}'); + return sb.toString(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java new file mode 100644 index 0000000000000..56c3cd2cc756f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -0,0 +1,99 @@ +/* + * 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.metadata; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +/** + * Interface that supports querying various pieces of metadata about a hudi table. + */ +public interface HoodieTableMetadata extends Serializable, AutoCloseable { + + // Table name suffix + String METADATA_TABLE_NAME_SUFFIX = "_metadata"; + /** + * Timestamp for a commit when the base dataset had not had any commits yet. this is < than even + * {@link org.apache.hudi.common.table.timeline.HoodieTimeline#INIT_INSTANT_TS}, such that the metadata table + * can be prepped even before bootstrap is done. + */ + String SOLO_COMMIT_TIMESTAMP = "0000000000000"; + // Key for the record which saves list of all partitions + String RECORDKEY_PARTITION_LIST = "__all_partitions__"; + // The partition name used for non-partitioned tables + String NON_PARTITIONED_NAME = "."; + + // Base path of the Metadata Table relative to the dataset (.hoodie/metadata) + static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata"; + + /** + * Return the base path of the Metadata Table. + * + * @param tableBasePath The base path of the dataset + */ + static String getMetadataTableBasePath(String tableBasePath) { + return tableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH; + } + + /** + * Returns {@code True} if the given path contains a metadata table. + * + * @param basePath The base path to check + */ + static boolean isMetadataTable(String basePath) { + return basePath.endsWith(METADATA_TABLE_REL_PATH); + } + + static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, + String spillableMapPath) { + if (metadataConfig.useFileListingMetadata()) { + return new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, spillableMapPath); + } else { + return new FileSystemBackedTableMetadata(engineContext, new SerializableConfiguration(engineContext.getHadoopConf()), + datasetBasePath, metadataConfig.shouldAssumeDatePartitioning()); + } + } + + /** + * Fetch all the files at the given partition path, per the latest snapshot of the metadata. + */ + FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException; + + /** + * Fetch list of all partition paths, per the latest snapshot of the metadata. + */ + List getAllPartitionPaths() throws IOException; + + /** + * Get the instant time to which the metadata is synced w.r.t data timeline. + */ + Option getSyncedInstantTime(); + + boolean isInSync(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java new file mode 100644 index 0000000000000..594231225a6d7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -0,0 +1,343 @@ +/* + * 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.metadata; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanerPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; + +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; + +/** + * A utility to convert timeline information to metadata table records. + */ +public class HoodieTableMetadataUtil { + + private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class); + + /** + * Converts a timeline instant to metadata table records. + * + * @param datasetMetaClient The meta client associated with the timeline instant + * @param instant to fetch and convert to metadata table records + * @return a list of metadata table records + * @throws IOException + */ + public static Option> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient, HoodieInstant instant, Option lastSyncTs) throws IOException { + HoodieTimeline timeline = datasetMetaClient.getActiveTimeline(); + Option> records = Option.empty(); + ValidationUtils.checkArgument(instant.isCompleted(), "Only completed instants can be synced."); + + switch (instant.getAction()) { + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(datasetMetaClient, instant); + records = Option.of(convertMetadataToRecords(cleanMetadata, instant.getTimestamp())); + break; + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.COMPACTION_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + records = Option.of(convertMetadataToRecords(commitMetadata, instant.getTimestamp())); + break; + case HoodieTimeline.ROLLBACK_ACTION: + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + timeline.getInstantDetails(instant).get()); + records = Option.of(convertMetadataToRecords(rollbackMetadata, instant.getTimestamp(), lastSyncTs)); + break; + case HoodieTimeline.RESTORE_ACTION: + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + timeline.getInstantDetails(instant).get()); + records = Option.of(convertMetadataToRecords(restoreMetadata, instant.getTimestamp(), lastSyncTs)); + break; + case HoodieTimeline.SAVEPOINT_ACTION: + // Nothing to be done here + break; + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes( + timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + // Note: we only add new files created here. Replaced files are removed from metadata later by cleaner. + records = Option.of(convertMetadataToRecords(replaceMetadata, instant.getTimestamp())); + break; + default: + throw new HoodieException("Unknown type of action " + instant.getAction()); + } + + return records; + } + + /** + * Finds all new files/partitions created as part of commit and creates metadata table records for them. + * + * @param commitMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) { + List records = new LinkedList<>(); + List allPartitions = new LinkedList<>(); + commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { + final String partition = partitionStatName.equals("") ? NON_PARTITIONED_NAME : partitionStatName; + allPartitions.add(partition); + + Map newFiles = new HashMap<>(writeStats.size()); + writeStats.forEach(hoodieWriteStat -> { + String pathWithPartition = hoodieWriteStat.getPath(); + if (pathWithPartition == null) { + // Empty partition + LOG.warn("Unable to find path in write stat to update metadata table " + hoodieWriteStat); + return; + } + + int offset = partition.equals(NON_PARTITIONED_NAME) ? 0 : partition.length() + 1; + String filename = pathWithPartition.substring(offset); + ValidationUtils.checkState(!newFiles.containsKey(filename), "Duplicate files in HoodieCommitMetadata"); + newFiles.put(filename, hoodieWriteStat.getTotalWriteBytes()); + }); + + // New files added to a partition + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( + partition, Option.of(newFiles), Option.empty()); + records.add(record); + }); + + // New partitions created + HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(new ArrayList<>(allPartitions)); + records.add(record); + + LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType() + + ". #partitions_updated=" + records.size()); + return records; + } + + /** + * Finds all files that will be deleted as part of a planned clean and creates metadata table records for them. + * + * @param cleanerPlan from timeline to convert + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCleanerPlan cleanerPlan, String instantTime) { + List records = new LinkedList<>(); + + int[] fileDeleteCount = {0}; + cleanerPlan.getFilePathsToBeDeletedPerPartition().forEach((partition, deletedPathInfo) -> { + fileDeleteCount[0] += deletedPathInfo.size(); + + // Files deleted from a partition + List deletedFilenames = deletedPathInfo.stream().map(p -> new Path(p.getFilePath()).getName()) + .collect(Collectors.toList()); + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), + Option.of(deletedFilenames)); + records.add(record); + }); + + LOG.info("Found at " + instantTime + " from CleanerPlan. #partitions_updated=" + records.size() + + ", #files_deleted=" + fileDeleteCount[0]); + return records; + } + + /** + * Finds all files that were deleted as part of a clean and creates metadata table records for them. + * + * @param cleanMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) { + List records = new LinkedList<>(); + int[] fileDeleteCount = {0}; + + cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { + // Files deleted from a partition + List deletedFiles = partitionMetadata.getSuccessDeleteFiles(); + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), + Option.of(new ArrayList<>(deletedFiles))); + + records.add(record); + fileDeleteCount[0] += deletedFiles.size(); + }); + + LOG.info("Updating at " + instantTime + " from Clean. #partitions_updated=" + records.size() + + ", #files_deleted=" + fileDeleteCount[0]); + return records; + } + + /** + * Aggregates all files deleted and appended to from all rollbacks associated with a restore operation then + * creates metadata table records for them. + * + * @param restoreMetadata + * @param instantTime + * @return a list of metadata table records + */ + public static List convertMetadataToRecords(HoodieRestoreMetadata restoreMetadata, String instantTime, Option lastSyncTs) { + Map> partitionToAppendedFiles = new HashMap<>(); + Map> partitionToDeletedFiles = new HashMap<>(); + restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { + rms.forEach(rm -> processRollbackMetadata(rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)); + }); + + return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"); + } + + public static List convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, Option lastSyncTs) { + + Map> partitionToAppendedFiles = new HashMap<>(); + Map> partitionToDeletedFiles = new HashMap<>(); + processRollbackMetadata(rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs); + return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback"); + } + + /** + * Extracts information about the deleted and append files from the {@code HoodieRollbackMetadata}. + * + * During a rollback files may be deleted (COW, MOR) or rollback blocks be appended (MOR only) to files. This + * function will extract this change file for each partition. + * + * @param rollbackMetadata {@code HoodieRollbackMetadata} + * @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition. + * @param partitionToAppendedFiles The {@code Map} to fill with files appended per partition and their sizes. + */ + private static void processRollbackMetadata(HoodieRollbackMetadata rollbackMetadata, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + Option lastSyncTs) { + + rollbackMetadata.getPartitionMetadata().values().forEach(pm -> { + // Has this rollback produced new files? + boolean hasRollbackLogFiles = pm.getRollbackLogFiles() != null && !pm.getRollbackLogFiles().isEmpty(); + boolean hasNonZeroRollbackLogFiles = hasRollbackLogFiles && pm.getRollbackLogFiles().values().stream().mapToLong(Long::longValue).sum() > 0; + // If commit being rolled back has not been synced to metadata table yet then there is no need to update metadata + boolean shouldSkip = lastSyncTs.isPresent() + && HoodieTimeline.compareTimestamps(rollbackMetadata.getCommitsRollback().get(0), HoodieTimeline.GREATER_THAN, lastSyncTs.get()); + + if (!hasNonZeroRollbackLogFiles && shouldSkip) { + LOG.info(String.format("Skipping syncing of rollbackMetadata at %s, given metadata table is already synced upto to %s", + rollbackMetadata.getCommitsRollback().get(0), lastSyncTs.get())); + return; + } + + final String partition = pm.getPartitionPath(); + if (!pm.getSuccessDeleteFiles().isEmpty() && !shouldSkip) { + if (!partitionToDeletedFiles.containsKey(partition)) { + partitionToDeletedFiles.put(partition, new ArrayList<>()); + } + + // Extract deleted file name from the absolute paths saved in getSuccessDeleteFiles() + List deletedFiles = pm.getSuccessDeleteFiles().stream().map(p -> new Path(p).getName()) + .collect(Collectors.toList()); + partitionToDeletedFiles.get(partition).addAll(deletedFiles); + } + + BiFunction fileMergeFn = (oldSize, newSizeCopy) -> { + // if a file exists in both written log files and rollback log files, we want to pick the one that is higher + // as rollback file could have been updated after written log files are computed. + return oldSize > newSizeCopy ? oldSize : newSizeCopy; + }; + + if (hasRollbackLogFiles) { + if (!partitionToAppendedFiles.containsKey(partition)) { + partitionToAppendedFiles.put(partition, new HashMap<>()); + } + + // Extract appended file name from the absolute paths saved in getAppendFiles() + pm.getRollbackLogFiles().forEach((path, size) -> { + partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, fileMergeFn); + }); + } + + if (pm.getWrittenLogFiles() != null && !pm.getWrittenLogFiles().isEmpty()) { + if (!partitionToAppendedFiles.containsKey(partition)) { + partitionToAppendedFiles.put(partition, new HashMap<>()); + } + + // Extract appended file name from the absolute paths saved in getWrittenLogFiles() + pm.getWrittenLogFiles().forEach((path, size) -> { + partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, fileMergeFn); + }); + } + }); + } + + private static List convertFilesToRecords(Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, String instantTime, + String operation) { + List records = new LinkedList<>(); + int[] fileChangeCount = {0, 0}; // deletes, appends + + partitionToDeletedFiles.forEach((partition, deletedFiles) -> { + fileChangeCount[0] += deletedFiles.size(); + + Option> filesAdded = Option.empty(); + if (partitionToAppendedFiles.containsKey(partition)) { + filesAdded = Option.of(partitionToAppendedFiles.remove(partition)); + } + + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, filesAdded, + Option.of(new ArrayList<>(deletedFiles))); + records.add(record); + }); + + partitionToAppendedFiles.forEach((partition, appendedFileMap) -> { + fileChangeCount[1] += appendedFileMap.size(); + + // Validate that no appended file has been deleted + ValidationUtils.checkState( + !appendedFileMap.keySet().removeAll(partitionToDeletedFiles.getOrDefault(partition, Collections.emptyList())), + "Rollback file cannot both be appended and deleted"); + + // New files added to a partition + HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.of(appendedFileMap), + Option.empty()); + records.add(record); + }); + + LOG.info("Found at " + instantTime + " from " + operation + ". #partitions_updated=" + records.size() + + ", #files_deleted=" + fileChangeCount[0] + ", #files_appended=" + fileChangeCount[1]); + + return records; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java new file mode 100644 index 0000000000000..0436de707d2fd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataPartitionType.java @@ -0,0 +1,33 @@ +/* + * 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.metadata; + +public enum MetadataPartitionType { + FILES("files"); + + private final String partitionPath; + + MetadataPartitionType(String partitionPath) { + this.partitionPath = partitionPath; + } + + public String partitionPath() { + return partitionPath; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java new file mode 100644 index 0000000000000..9ba3f26079d14 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/TimelineMergedTableMetadata.java @@ -0,0 +1,115 @@ +/* + * 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.metadata; + +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.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** + * Provides functionality to convert timeline instants to table metadata records and then merge by key. Specify + * a filter to limit keys that are merged and stored in memory. + */ +public class TimelineMergedTableMetadata implements Serializable { + + private static final Logger LOG = LogManager.getLogger(TimelineMergedTableMetadata.class); + + HoodieTableMetaClient metaClient; + private List instants; + private Option lastSyncTs; + private Set mergeKeyFilter; + + // keep it a simple hash map, so it can be easily passed onto the executors, once merged. + protected final Map> timelineMergedRecords; + + public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, List instants, + Option lastSyncTs, Set mergeKeyFilter) { + this.metaClient = metaClient; + this.instants = instants; + this.lastSyncTs = lastSyncTs; + this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet(); + this.timelineMergedRecords = new HashMap<>(); + + scan(); + } + + /** + * Converts instants in scanner to metadata table records and processes each record. + * + * @param + * @throws IOException + */ + private void scan() { + for (HoodieInstant instant : instants) { + try { + Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient, instant, lastSyncTs); + if (records.isPresent()) { + records.get().forEach(record -> processNextRecord(record)); + } + } catch (Exception e) { + LOG.error(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); + throw new HoodieException(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e); + } + } + } + + /** + * Process metadata table record by merging with existing record if it is a part of the key filter. + * + * @param hoodieRecord + */ + private void processNextRecord(HoodieRecord hoodieRecord) { + String key = hoodieRecord.getRecordKey(); + if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(key)) { + if (timelineMergedRecords.containsKey(key)) { + // Merge and store the merged record + HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(timelineMergedRecords.get(key).getData(), new Properties()); + timelineMergedRecords.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue)); + } else { + // Put the record as is + timelineMergedRecords.put(key, hoodieRecord); + } + } + } + + /** + * Retrieve merged hoodie record for given key. + * + * @param key of the record to retrieve + * @return {@code HoodieRecord} if key was found else {@code Option.empty()} + */ + public Option> getRecordByKey(String key) { + return Option.ofNullable((HoodieRecord) timelineMergedRecords.get(key)); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index 40db67b50870e..863103ebf2c59 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -207,4 +207,34 @@ public void testAddingAndRemovingMetadataFields() { Schema schemaWithoutMetaCols = HoodieAvroUtils.removeMetadataFields(schemaWithMetaCols); assertEquals(schemaWithoutMetaCols.getFields().size(), NUM_FIELDS_IN_EXAMPLE_SCHEMA); } + + @Test + public void testGetNestedFieldVal() { + GenericRecord rec = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA)); + rec.put("_row_key", "key1"); + rec.put("non_pii_col", "val1"); + rec.put("pii_col", "val2"); + + Object rowKey = HoodieAvroUtils.getNestedFieldVal(rec, "_row_key", true); + assertEquals(rowKey, "key1"); + + Object rowKeyNotExist = HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", true); + assertNull(rowKeyNotExist); + + // Field does not exist + try { + HoodieAvroUtils.getNestedFieldVal(rec, "fake_key", false); + } catch (Exception e) { + assertEquals("fake_key(Part -fake_key) field not found in record. Acceptable fields were :[timestamp, _row_key, non_pii_col, pii_col]", + e.getMessage()); + } + + // Field exist while value not + try { + HoodieAvroUtils.getNestedFieldVal(rec, "timestamp", false); + } catch (Exception e) { + assertEquals("The value of timestamp can not be null", e.getMessage()); + } + } + } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index fa25e17755c1f..57e814c339090 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.log.AppendResult; import org.apache.hudi.common.table.log.HoodieLogFileReader; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Reader; @@ -76,6 +77,7 @@ import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -119,7 +121,7 @@ public void tearDown() throws IOException { } @Test - public void testEmptyLog() throws IOException, InterruptedException { + public void testEmptyLog() throws IOException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); @@ -138,18 +140,21 @@ public void testBasicAppend(HoodieLogBlockType dataBlockType) throws IOException Map header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); + long pos = writer.getCurrentSize(); HoodieDataBlock dataBlock = getDataBlock(dataBlockType, records, header); - writer = writer.appendBlock(dataBlock); + AppendResult result = writer.appendBlock(dataBlock); + long size = writer.getCurrentSize(); assertTrue(size > 0, "We just wrote a block - size should be > 0"); assertEquals(size, fs.getFileStatus(writer.getLogFile().getPath()).getLen(), "Write should be auto-flushed. The size reported by FileStatus and the writer should match"); + assertEquals(size, result.size()); + assertEquals(writer.getLogFile(), result.logFile()); + assertEquals(0, result.offset()); writer.close(); - } - @ParameterizedTest - @EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" }) + @Test public void testRollover() throws IOException, InterruptedException, URISyntaxException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) @@ -160,23 +165,36 @@ public void testRollover() throws IOException, InterruptedException, URISyntaxEx header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); // Write out a block - writer = writer.appendBlock(dataBlock); + AppendResult firstAppend = writer.appendBlock(dataBlock); // Get the size of the block long size = writer.getCurrentSize(); writer.close(); + assertEquals(0, firstAppend.offset()); + assertEquals(size, firstAppend.size()); + // Create a writer with the size threshold as the size we just wrote - so this has to roll writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build(); records = SchemaTestUtil.generateTestRecords(0, 100); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + AppendResult secondAppend = writer.appendBlock(dataBlock); + + assertEquals(firstAppend.logFile(), secondAppend.logFile()); + assertNotEquals(0, secondAppend.offset()); assertEquals(0, writer.getCurrentSize(), "This should be a new log file and hence size should be 0"); assertEquals(2, writer.getLogFile().getLogVersion(), "Version should be rolled to 2"); Path logFilePath = writer.getLogFile().getPath(); assertFalse(fs.exists(logFilePath), "Path (" + logFilePath + ") must not exist"); + + // Write one more block, which should not go to the new log file. + records = SchemaTestUtil.generateTestRecords(0, 100); + dataBlock = getDataBlock(records, header); + AppendResult rolloverAppend = writer.appendBlock(dataBlock); + + assertNotEquals(secondAppend.logFile(), rolloverAppend.logFile()); + assertEquals(0, rolloverAppend.offset()); writer.close(); } @@ -203,17 +221,13 @@ private void testConcurrentAppend(boolean logFileExists, boolean newLogFileForma if (newLogFileFormat && logFileExists) { // Assume there is an existing log-file with write token - builder1 = builder1.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN) - .withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); - builder2 = builder2.withLogVersion(1).withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN) - .withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); + builder1 = builder1.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); + builder2 = builder2.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); } else if (newLogFileFormat) { // First log file of the file-slice builder1 = builder1.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) - .withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN) .withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); builder2 = builder2.withLogVersion(HoodieLogFile.LOGFILE_BASE_VERSION) - .withLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN) .withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); } else { builder1 = builder1.withLogVersion(1).withRolloverLogWriteToken(HoodieLogFormat.UNKNOWN_WRITE_TOKEN); @@ -224,9 +238,9 @@ private void testConcurrentAppend(boolean logFileExists, boolean newLogFileForma header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); Writer writer2 = builder2.build(); - writer2 = writer2.appendBlock(dataBlock); + writer2.appendBlock(dataBlock); HoodieLogFile logFile1 = writer.getLogFile(); HoodieLogFile logFile2 = writer2.getLogFile(); writer.close(); @@ -245,7 +259,7 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); long size1 = writer.getCurrentSize(); writer.close(); @@ -255,7 +269,7 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); long size2 = writer.getCurrentSize(); assertTrue(size2 > size1, "We just wrote a new block - size2 should be > size1"); assertEquals(size2, fs.getFileStatus(writer.getLogFile().getPath()).getLen(), @@ -269,7 +283,7 @@ public void testMultipleAppend() throws IOException, URISyntaxException, Interru records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); long size3 = writer.getCurrentSize(); assertTrue(size3 > size2, "We just wrote a new block - size3 should be > size2"); assertEquals(size3, fs.getFileStatus(writer.getLogFile().getPath()).getLen(), @@ -325,9 +339,11 @@ public void testAppendNotSupported() throws IOException, URISyntaxException, Int HoodieDataBlock dataBlock = getDataBlock(records, header); for (int i = 0; i < 2; i++) { - HoodieLogFormat.newWriterBuilder().onParentPath(testPath) + Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("") - .withFs(localFs).build().appendBlock(dataBlock).close(); + .withFs(localFs).build(); + writer.appendBlock(dataBlock); + writer.close(); } // ensure there are two log file versions, with same data. @@ -335,8 +351,7 @@ public void testAppendNotSupported() throws IOException, URISyntaxException, Int assertEquals(2, statuses.length); } - @ParameterizedTest - @EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" }) + @Test public void testBasicWriteAndScan() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) @@ -349,7 +364,7 @@ public void testBasicWriteAndScan() throws IOException, URISyntaxException, Inte header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); @@ -364,8 +379,7 @@ public void testBasicWriteAndScan() throws IOException, URISyntaxException, Inte reader.close(); } - @ParameterizedTest - @EnumSource(names = { "AVRO_DATA_BLOCK", "HFILE_DATA_BLOCK" }) + @Test public void testBasicAppendAndRead() throws IOException, URISyntaxException, InterruptedException { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) @@ -378,7 +392,7 @@ public void testBasicAppendAndRead() throws IOException, URISyntaxException, Int header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); writer = @@ -389,7 +403,7 @@ public void testBasicAppendAndRead() throws IOException, URISyntaxException, Int .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records @@ -401,7 +415,7 @@ public void testBasicAppendAndRead() throws IOException, URISyntaxException, Int .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records3, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); Reader reader = HoodieLogFormat.newReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema()); @@ -455,14 +469,25 @@ public void testBasicAppendAndScanMultipleFiles(boolean readBlocksLazily) allRecords.add(copyOfRecords1); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); } writer.close(); // scan all log blocks (across multiple log files) - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, - logFiles.stream().map(logFile -> logFile.getPath().toString()).collect(Collectors.toList()), schema, "100", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths( + logFiles.stream() + .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList())) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); List scannedRecords = new ArrayList<>(); for (HoodieRecord record : scanner) { @@ -484,7 +509,7 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) @@ -510,7 +535,7 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep records = SchemaTestUtil.generateTestRecords(0, 10); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // First round of reads - we should be able to read the first block and then EOF @@ -548,7 +573,7 @@ public void testAppendAndReadOnCorruptedLog() throws IOException, URISyntaxExcep records = SchemaTestUtil.generateTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString()); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Second round of reads - we should be able to read the first and last block @@ -586,7 +611,7 @@ public void testAvroLogRecordReaderBasic(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); @@ -594,15 +619,25 @@ public void testAvroLogRecordReaderBasic(boolean readBlocksLazily) .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(200, scanner.getTotalLogRecords()); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -633,21 +668,21 @@ public void testAvroLogRecordReaderWithRollbackTombstone(boolean readBlocksLazil header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Write 2 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Rollback the last write header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); @@ -656,15 +691,25 @@ public void testAvroLogRecordReaderWithRollbackTombstone(boolean readBlocksLazil .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = getDataBlock(records3, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("102") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches"); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -693,7 +738,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Write 2 @@ -725,7 +770,7 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() writer = HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); // Write 3 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "103"); @@ -735,15 +780,25 @@ public void testAvroLogRecordReaderWithRollbackPartialBlock() header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = getDataBlock(records3, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "103", - 10240L, true, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("103") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(true) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records"); Set readKeys = new HashSet<>(200); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -773,7 +828,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Write 2 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); @@ -781,7 +836,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil List copyOfRecords2 = records2.stream() .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); copyOfRecords1.addAll(copyOfRecords2); List originalKeys = @@ -796,14 +851,24 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "102"); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); - writer = writer.appendBlock(deleteBlock); + writer.appendBlock(deleteBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "102", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("102") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records"); final List readKeys = new ArrayList<>(200); final List emptyPayloads = new ArrayList<>(); @@ -833,8 +898,18 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(boolean readBlocksLazil writer.appendBlock(commandBlock); readKeys.clear(); - scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", 10240L, readBlocksLazily, - false, bufferSize, BASE_OUTPUT_PATH); + scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("101") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); assertEquals(200, readKeys.size(), "Stream collect should return all 200 records after rollback of delete"); } @@ -861,13 +936,13 @@ public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Write 2 List records2 = SchemaTestUtil.generateHoodieTestRecords(0, 100); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Delete 50 keys // Delete 50 keys @@ -877,14 +952,14 @@ public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily) .collect(Collectors.toList()).subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); - writer = writer.appendBlock(deleteBlock); + writer.appendBlock(deleteBlock); // Attempt 1 : Write rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); try { - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); // Say job failed, retry writing 2 rollback in the next rollback(..) attempt throw new Exception("simulating failure"); } catch (Exception e) { @@ -898,8 +973,18 @@ public void testAvroLogRecordReaderWithFailedRollbacks(boolean readBlocksLazily) .map(s -> s.getPath().toString()).collect(Collectors.toList()); // all data must be rolled back before merge - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback"); final List readKeys = new ArrayList<>(); @@ -928,7 +1013,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(boolean readBlock header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Delete 50 keys List deletedKeys = copyOfRecords1.stream() @@ -936,21 +1021,31 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(boolean readBlock ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) .collect(Collectors.toList()).subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); - writer = writer.appendBlock(deleteBlock); + writer.appendBlock(deleteBlock); // Write 2 rollback blocks (1 data block + 1 delete bloc) for a failed write header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); writer.appendBlock(commandBlock); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); } @@ -970,7 +1065,7 @@ public void testAvroLogRecordReaderWithInvalidRollback(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Write invalid rollback for a failed write (possible for in-flight commits) header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); @@ -983,8 +1078,18 @@ public void testAvroLogRecordReaderWithInvalidRollback(boolean readBlocksLazily) FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "100", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records"); final List readKeys = new ArrayList<>(100); scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); @@ -1012,9 +1117,9 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(boolean readBloc header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); - writer = writer.appendBlock(dataBlock); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Delete 50 keys // Delete 50 keys @@ -1023,7 +1128,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(boolean readBloc ((GenericRecord) s).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString()))) .collect(Collectors.toList()).subList(0, 50); HoodieDeleteBlock deleteBlock = new HoodieDeleteBlock(deletedKeys.toArray(new HoodieKey[50]), header); - writer = writer.appendBlock(deleteBlock); + writer.appendBlock(deleteBlock); // Write 1 rollback block for a failed write header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); @@ -1036,8 +1141,18 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(boolean readBloc FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("101") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); } @@ -1059,9 +1174,9 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean r header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); - writer = writer.appendBlock(dataBlock); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Append some arbit byte[] to the end of the log (mimics a partially written commit) @@ -1094,7 +1209,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean r HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Append some arbit byte[] to the end of the log (mimics a partially written commit) @@ -1119,15 +1234,25 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(boolean r header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(commandBlock); + writer.appendBlock(commandBlock); writer.close(); List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") .map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, "101", - 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("101") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); } @@ -1161,7 +1286,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records.subList(0, numRecordsInLog1), header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // Get the size of the block long size = writer.getCurrentSize(); writer.close(); @@ -1175,7 +1300,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock2 = getDataBlock(records2.subList(0, numRecordsInLog2), header2); - writer2 = writer2.appendBlock(dataBlock2); + writer2.appendBlock(dataBlock2); // Get the size of the block writer2.close(); @@ -1183,8 +1308,18 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 List allLogFiles = FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100").map(s -> s.getPath().toString()).collect(Collectors.toList()); - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, basePath, allLogFiles, schema, - "100", 10240L, readBlocksLazily, false, bufferSize, BASE_OUTPUT_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(bufferSize) + .withSpillableMapBasePath(BASE_OUTPUT_PATH) + .build(); assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(), "We would read 100 records"); @@ -1239,7 +1374,7 @@ public void testBasicAppendAndReadInReverse(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); writer = @@ -1249,7 +1384,7 @@ public void testBasicAppendAndReadInReverse(boolean readBlocksLazily) List copyOfRecords2 = records2.stream() .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records @@ -1260,7 +1395,7 @@ public void testBasicAppendAndReadInReverse(boolean readBlocksLazily) List copyOfRecords3 = records3.stream() .map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList()); dataBlock = getDataBlock(records3, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), @@ -1308,7 +1443,7 @@ public void testAppendAndReadOnCorruptedLogInReverse(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Append some arbit byte[] to thee end of the log (mimics a partially written commit) @@ -1334,7 +1469,7 @@ public void testAppendAndReadOnCorruptedLogInReverse(boolean readBlocksLazily) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); records = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = getDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // First round of reads - we should be able to read the first block and then EOF @@ -1367,7 +1502,7 @@ public void testBasicAppendAndTraverseInReverse(boolean readBlocksLazily) header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); HoodieDataBlock dataBlock = getDataBlock(records1, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); writer = @@ -1375,7 +1510,7 @@ public void testBasicAppendAndTraverseInReverse(boolean readBlocksLazily) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); List records2 = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = getDataBlock(records2, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); // Close and Open again and append 100 more records @@ -1384,7 +1519,7 @@ public void testBasicAppendAndTraverseInReverse(boolean readBlocksLazily) .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); List records3 = SchemaTestUtil.generateTestRecords(0, 100); dataBlock = getDataBlock(records3, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); writer.close(); HoodieLogFileReader reader = new HoodieLogFileReader(fs, writer.getLogFile(), SchemaTestUtil.getSimpleSchema(), diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java index 71616f6683eda..e313bb4a6ca0f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java @@ -110,7 +110,7 @@ public void testFailedToGetAppendStreamFromHDFSNameNode() .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") .overBaseCommit("").withFs(fs).build(); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); // get the current log file version to compare later int logFileVersion = writer.getLogFile().getLogVersion(); Path logFilePath = writer.getLogFile().getPath(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java new file mode 100644 index 0000000000000..791415428e476 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Type; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** + * Unit tests {@link DefaultHoodieRecordPayload}. + */ +public class TestDefaultHoodieRecordPayload { + + private Schema schema; + private Properties props; + + @BeforeEach + public void setUp() throws Exception { + schema = Schema.createRecord(Arrays.asList( + new Schema.Field("id", Schema.create(Schema.Type.STRING), "", null), + new Schema.Field("partition", Schema.create(Schema.Type.STRING), "", null), + new Schema.Field("ts", Schema.create(Schema.Type.LONG), "", null), + new Schema.Field("_hoodie_is_deleted", Schema.create(Type.BOOLEAN), "", false) + )); + props = new Properties(); + props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "ts"); + } + + @Test + public void testActiveRecords() throws IOException { + GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", "1"); + record1.put("partition", "partition0"); + record1.put("ts", 0L); + record1.put("_hoodie_is_deleted", false); + + GenericRecord record2 = new GenericData.Record(schema); + record2.put("id", "2"); + record2.put("partition", "partition1"); + record2.put("ts", 1L); + record2.put("_hoodie_is_deleted", false); + + DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1); + DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(record2, 2); + assertEquals(payload1.preCombine(payload2, props), payload2); + assertEquals(payload2.preCombine(payload1, props), payload2); + + assertEquals(record1, payload1.getInsertValue(schema).get()); + assertEquals(record2, payload2.getInsertValue(schema).get()); + + assertEquals(payload1.combineAndGetUpdateValue(record2, schema, props).get(), record2); + assertEquals(payload2.combineAndGetUpdateValue(record1, schema, props).get(), record2); + } + + @Test + public void testDeletedRecord() throws IOException { + GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", "1"); + record1.put("partition", "partition0"); + record1.put("ts", 0L); + record1.put("_hoodie_is_deleted", false); + + GenericRecord delRecord1 = new GenericData.Record(schema); + delRecord1.put("id", "2"); + delRecord1.put("partition", "partition1"); + delRecord1.put("ts", 1L); + delRecord1.put("_hoodie_is_deleted", true); + + DefaultHoodieRecordPayload payload1 = new DefaultHoodieRecordPayload(record1, 1); + DefaultHoodieRecordPayload payload2 = new DefaultHoodieRecordPayload(delRecord1, 2); + assertEquals(payload1.preCombine(payload2, props), payload2); + assertEquals(payload2.preCombine(payload1, props), payload2); + + assertEquals(record1, payload1.getInsertValue(schema).get()); + assertFalse(payload2.getInsertValue(schema).isPresent()); + + assertEquals(payload1.combineAndGetUpdateValue(delRecord1, schema, props).get(), delRecord1); + assertFalse(payload2.combineAndGetUpdateValue(record1, schema, props).isPresent()); + } + +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java new file mode 100644 index 0000000000000..bcd27f37591e3 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java @@ -0,0 +1,50 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.ArrayList; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests hoodie delta write stat {@link HoodieDeltaWriteStat}. + */ +public class TestHoodieDeltaWriteStat { + + @Test + public void testBaseFileAndLogFiles() { + HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat(); + String baseFile = "file1.parquet"; + String logFile1 = ".log1.log"; + String logFile2 = ".log2.log"; + + writeStat.setBaseFile(baseFile); + writeStat.addLogFiles(logFile1); + writeStat.addLogFiles(logFile2); + assertTrue(writeStat.getLogFiles().contains(logFile1)); + assertTrue(writeStat.getLogFiles().contains(logFile2)); + assertEquals(baseFile, writeStat.getBaseFile()); + + writeStat.setLogFiles(new ArrayList<>()); + assertTrue(writeStat.getLogFiles().isEmpty()); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java index e677f491f14d5..18c0d3f20a659 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java @@ -23,8 +23,10 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.HoodieRollbackStat; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -58,6 +60,43 @@ public void setUp() throws Exception { initMetaClient(); } + @Test + public void testGetPartitionsWithReplaceCommits() throws IOException { + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + HoodieTimeline activeCommitTimeline = activeTimeline.getCommitTimeline(); + assertTrue(activeCommitTimeline.empty()); + + String ts1 = "1"; + String replacePartition = "2021/01/01"; + String newFilePartition = "2021/01/02"; + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, ts1); + activeTimeline.createNewInstant(instant1); + // create replace metadata only with replaced file Ids (no new files created) + activeTimeline.saveAsComplete(instant1, + Option.of(getReplaceCommitMetadata(basePath, ts1, replacePartition, 2, newFilePartition, 0, Collections.emptyMap()))); + metaClient.reloadActiveTimeline(); + + List partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsAfter("0", 10)); + assertEquals(1, partitions.size()); + assertEquals(replacePartition, partitions.get(0)); + + String ts2 = "2"; + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, ts2); + activeTimeline.createNewInstant(instant2); + // create replace metadata only with replaced file Ids (no new files created) + activeTimeline.saveAsComplete(instant2, + Option.of(getReplaceCommitMetadata(basePath, ts2, replacePartition, 0, newFilePartition, 3, Collections.emptyMap()))); + metaClient.reloadActiveTimeline(); + partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsAfter("1", 10)); + assertEquals(1, partitions.size()); + assertEquals(newFilePartition, partitions.get(0)); + + partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsAfter("0", 10)); + assertEquals(2, partitions.size()); + assertTrue(partitions.contains(replacePartition)); + assertTrue(partitions.contains(newFilePartition)); + } + @Test public void testGetPartitions() throws IOException { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); @@ -81,20 +120,20 @@ public void testGetPartitions() throws IOException { // verify modified partitions included cleaned data List partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsAfter("1", 10)); assertEquals(5, partitions.size()); - assertEquals(partitions, Arrays.asList(new String[]{"0", "2", "3", "4", "5"})); + assertEquals(partitions, Arrays.asList(new String[] {"0", "2", "3", "4", "5"})); partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsInRange("1", "4")); assertEquals(4, partitions.size()); - assertEquals(partitions, Arrays.asList(new String[]{"0", "2", "3", "4"})); + assertEquals(partitions, Arrays.asList(new String[] {"0", "2", "3", "4"})); // verify only commit actions partitions = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().findInstantsAfter("1", 10)); assertEquals(4, partitions.size()); - assertEquals(partitions, Arrays.asList(new String[]{"2", "3", "4", "5"})); + assertEquals(partitions, Arrays.asList(new String[] {"2", "3", "4", "5"})); partitions = TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().findInstantsInRange("1", "4")); assertEquals(3, partitions.size()); - assertEquals(partitions, Arrays.asList(new String[]{"2", "3", "4"})); + assertEquals(partitions, Arrays.asList(new String[] {"2", "3", "4"})); } @Test @@ -142,10 +181,10 @@ public void testRestoreInstants() throws Exception { // verify modified partitions included cleaned data List partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsAfter("1", 10)); - assertEquals(partitions, Arrays.asList(new String[]{"2", "3", "4", "5"})); + assertEquals(partitions, Arrays.asList(new String[] {"2", "3", "4", "5"})); partitions = TimelineUtils.getAffectedPartitions(metaClient.getActiveTimeline().findInstantsInRange("1", "4")); - assertEquals(partitions, Arrays.asList(new String[]{"2", "3", "4"})); + assertEquals(partitions, Arrays.asList(new String[] {"2", "3", "4"})); } @Test @@ -162,7 +201,7 @@ public void testGetExtraMetadata() throws Exception { activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, ts, ts, 2, Collections.emptyMap()))); - ts = "1"; + ts = "1"; instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, ts); activeTimeline.createNewInstant(instant); Map extraMetadata = new HashMap<>(); @@ -202,7 +241,8 @@ private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, Stri List rollbacks = new ArrayList<>(); rollbacks.add(new HoodieInstant(false, actionType, commitTs)); - HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap()); + HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap(), + Collections.EMPTY_MAP); List rollbackStats = new ArrayList<>(); rollbackStats.add(rollbackStat); return TimelineMetadataUtils.convertRollbackMetadata(commitTs, Option.empty(), rollbacks, rollbackStats); @@ -224,6 +264,31 @@ private byte[] getCommitMetadata(String basePath, String partition, String commi return commit.toJsonString().getBytes(StandardCharsets.UTF_8); } + private byte[] getReplaceCommitMetadata(String basePath, String commitTs, String replacePartition, int replaceCount, + String newFilePartition, int newFileCount, Map extraMetadata) + throws IOException { + HoodieReplaceCommitMetadata commit = new HoodieReplaceCommitMetadata(); + for (int i = 1; i <= newFileCount; i++) { + HoodieWriteStat stat = new HoodieWriteStat(); + stat.setFileId(i + ""); + stat.setPartitionPath(Paths.get(basePath, newFilePartition).toString()); + stat.setPath(commitTs + "." + i + ".parquet"); + commit.addWriteStat(newFilePartition, stat); + } + Map> partitionToReplaceFileIds = new HashMap<>(); + if (replaceCount > 0) { + partitionToReplaceFileIds.put(replacePartition, new ArrayList<>()); + } + for (int i = 1; i <= replaceCount; i++) { + partitionToReplaceFileIds.get(replacePartition).add(FSUtils.createNewFileIdPfx()); + } + commit.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + for (Map.Entry extraEntries : extraMetadata.entrySet()) { + commit.addMetadata(extraEntries.getKey(), extraEntries.getValue()); + } + return commit.toJsonString().getBytes(StandardCharsets.UTF_8); + } + private Option getCleanMetadata(String partition, String time) throws IOException { Map partitionToFilesCleaned = new HashMap<>(); List filesDeleted = new ArrayList<>(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 3fceee3bb40c1..e103427d40728 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -1356,6 +1356,13 @@ public void testReplaceWithTimeTravel() throws IOException { List allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList()); assertEquals(1, allReplaced.size()); assertEquals(fileId1, allReplaced.get(0).getFileGroupId().getFileId()); + + allReplaced = fsView.getReplacedFileGroupsBefore("2", partitionPath1).collect(Collectors.toList()); + assertEquals(0, allReplaced.size()); + + allReplaced = fsView.getAllReplacedFileGroups(partitionPath1).collect(Collectors.toList()); + assertEquals(1, allReplaced.size()); + assertEquals(fileId1, allReplaced.get(0).getFileGroupId().getFileId()); } @Test diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index e4933cf4e983b..146e0bb85b53d 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.HoodieCleanStat; @@ -35,6 +36,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; 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; @@ -554,7 +556,7 @@ private void performRestore(HoodieInstant instant, List files, String ro boolean isRestore) throws IOException { Map> partititonToFiles = deleteFiles(files); List rollbackStats = partititonToFiles.entrySet().stream().map(e -> - new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>()) + new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>(), new HashMap<>()) ).collect(Collectors.toList()); List rollbacks = new ArrayList<>(); @@ -857,11 +859,20 @@ private List addInstant(HoodieTableMetaClient metaClient, String instant private List addReplaceInstant(HoodieTableMetaClient metaClient, String instant, List> writeStats, Map> partitionToReplaceFileIds) throws IOException { + // created requested + HoodieInstant newRequestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instant); + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setOperationType(WriteOperationType.UNKNOWN.name()).build(); + metaClient.getActiveTimeline().saveToPendingReplaceCommit(newRequestedInstant, + TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + + metaClient.reloadActiveTimeline(); + // transition to inflight + HoodieInstant inflightInstant = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(newRequestedInstant, Option.empty()); + // transition to replacecommit HoodieReplaceCommitMetadata replaceCommitMetadata = new HoodieReplaceCommitMetadata(); writeStats.forEach(e -> replaceCommitMetadata.addWriteStat(e.getKey(), e.getValue())); replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds); - HoodieInstant inflightInstant = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, instant); - metaClient.getActiveTimeline().createNewInstant(inflightInstant); metaClient.getActiveTimeline().saveAsComplete(inflightInstant, Option.of(replaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList()); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java index 96a00da6f2b32..25b2c8baf9c7a 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java @@ -35,6 +35,7 @@ public class HoodieCommonTestHarness { protected String basePath = null; + protected transient HoodieTestDataGenerator dataGen = null; protected transient HoodieTableMetaClient metaClient; @TempDir public java.nio.file.Path tempDir; @@ -52,6 +53,24 @@ protected void initPath() { } } + /** + * Initializes a test data generator which used to generate test datas. + * + */ + protected void initTestDataGenerator() { + dataGen = new HoodieTestDataGenerator(); + } + + /** + * Cleanups test data generator. + * + */ + protected void cleanupTestDataGenerator() { + if (dataGen != null) { + dataGen = null; + } + } + /** * Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by * {@code getTableType()}. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index bf9728045e530..17e93feca95d2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -451,6 +451,10 @@ public List generateInsertsContainsAllPartitions(String instantTim return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, true).collect(Collectors.toList()); } + public List generateInsertsForPartition(String instantTime, Integer n, String partition) { + return generateInsertsStream(instantTime, n, false, TRIP_EXAMPLE_SCHEMA, false, () -> partition, () -> UUID.randomUUID().toString()).collect(Collectors.toList()); + } + public Stream generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) { return generateInsertsStream(commitTime, n, isFlattened, schemaStr, containsAllPartitions, () -> partitionPaths[RAND.nextInt(partitionPaths.length)], diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 3663917a54d75..858e113734b58 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -228,6 +228,11 @@ public HoodieTestTable forDeltaCommit(String instantTime) { return this; } + public HoodieTestTable forReplaceCommit(String instantTime) { + currentInstantTime = instantTime; + return this; + } + public HoodieTestTable forCompaction(String instantTime) { currentInstantTime = instantTime; return this; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java index 5d82bbce734ca..54ca072651e07 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java @@ -98,6 +98,22 @@ public void testClusteringPlanMultipleInstants() throws Exception { validateClusteringInstant(fileIds3, partitionPath1, clusterTime, fileGroupToInstantMap); } + // replacecommit.inflight doesnt have clustering plan. + // Verify that getClusteringPlan fetches content from corresponding requested file. + @Test + public void testClusteringPlanInflight() throws Exception { + String partitionPath1 = "partition1"; + List fileIds1 = new ArrayList<>(); + fileIds1.add(UUID.randomUUID().toString()); + fileIds1.add(UUID.randomUUID().toString()); + String clusterTime1 = "1"; + HoodieInstant requestedInstant = createRequestedReplaceInstant(partitionPath1, clusterTime1, fileIds1); + HoodieInstant inflightInstant = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant, Option.empty()); + HoodieClusteringPlan requestedClusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, requestedInstant).get().getRight(); + HoodieClusteringPlan inflightClusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, inflightInstant).get().getRight(); + assertEquals(requestedClusteringPlan, inflightClusteringPlan); + } + private void validateClusteringInstant(List fileIds, String partitionPath, String expectedInstantTime, Map fileGroupToInstantMap) { for (String fileId : fileIds) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java index 9496f0195b9c4..2bcbcbdab67a3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestParquetUtils.java @@ -36,6 +36,7 @@ import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -147,6 +148,18 @@ public void testFetchRecordKeyPartitionPathFromParquet(String typeCode) throws E } } + @Test + public void testReadCounts() throws Exception { + String filePath = basePath + "/test.parquet"; + List rowKeys = new ArrayList<>(); + for (int i = 0; i < 123; i++) { + rowKeys.add(UUID.randomUUID().toString()); + } + writeParquetFile(BloomFilterTypeCode.SIMPLE.name(), filePath, rowKeys); + + assertEquals(123, ParquetUtils.getRowCount(HoodieTestUtils.getDefaultHadoopConf(), new Path(filePath))); + } + private void writeParquetFile(String typeCode, String filePath, List rowKeys) throws Exception { writeParquetFile(typeCode, filePath, rowKeys, HoodieAvroUtils.getRecordKeySchema(), false, ""); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.java new file mode 100644 index 0000000000000..03421a3005f04 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.java @@ -0,0 +1,89 @@ +/* + * 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.util; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests serializable schema. + */ +public class TestSerializableSchema { + + @Test + public void testSerDeser() throws IOException { + verifySchema(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA)); + verifySchema(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA)); + verifySchema(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA)); + verifySchema(HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS); + } + + @Test + public void testLargeSchema() throws IOException { + verifySchema(new Schema.Parser().parse(generateLargeSchema())); + } + + private void verifySchema(Schema schema) throws IOException { + SerializableSchema serializableSchema = new SerializableSchema(schema); + assertEquals(schema, serializableSchema.get()); + assertTrue(schema != serializableSchema.get()); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + serializableSchema.writeObjectTo(oos); + oos.flush(); + oos.close(); + + byte[] bytesWritten = baos.toByteArray(); + SerializableSchema newSchema = new SerializableSchema(); + newSchema.readObjectFrom(new ObjectInputStream(new ByteArrayInputStream(bytesWritten))); + assertEquals(schema, newSchema.get()); + } + + // generate large schemas (>64K which is limitation of ObjectOutputStream#writeUTF) to validate it can be serialized + private String generateLargeSchema() { + StringBuilder schema = new StringBuilder(); + schema.append(HoodieTestDataGenerator.TRIP_SCHEMA_PREFIX); + int fieldNum = 1; + while (schema.length() < 80 * 1024) { + String fieldName = "field" + fieldNum; + schema.append("{\"name\": \"" + fieldName + "\",\"type\": {\"type\":\"record\", \"name\":\"" + fieldName + "\",\"fields\": [" + + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},"); + fieldNum++; + } + + schema.append(HoodieTestDataGenerator.TRIP_SCHEMA_SUFFIX); + return schema.toString(); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/TestFileSystemBackedTableMetadata.java b/hudi-common/src/test/java/org/apache/hudi/metadata/TestFileSystemBackedTableMetadata.java new file mode 100644 index 0000000000000..e44054f3aab36 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/TestFileSystemBackedTableMetadata.java @@ -0,0 +1,174 @@ +/* + * 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.metadata; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +public class TestFileSystemBackedTableMetadata extends HoodieCommonTestHarness { + + private static final String DEFAULT_PARTITION = ""; + private static final List DATE_PARTITIONS = Arrays.asList("2019/01/01", "2020/01/02", "2021/03/01"); + private static final List ONE_LEVEL_PARTITIONS = Arrays.asList("2019", "2020", "2021"); + private static final List MULTI_LEVEL_PARTITIONS = Arrays.asList("2019/01", "2020/01", "2021/01"); + private static HoodieTestTable hoodieTestTable; + + @BeforeEach + public void setUp() throws IOException { + initMetaClient(); + hoodieTestTable = HoodieTestTable.of(metaClient); + } + + @AfterEach + public void tearDown() throws IOException { + metaClient.getFs().delete(new Path(metaClient.getBasePath()), true); + } + + /** + * Test non partition hoodie table. + * @throws Exception + */ + @Test + public void testNonPartitionedTable() throws Exception { + // Generate 10 files under basepath + hoodieTestTable.addCommit("100").withBaseFilesInPartition(DEFAULT_PARTITION, IntStream.range(0, 10).toArray()); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, false); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 0); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(basePath)).length == 10); + } + + /** + * Test listing of partitions result for date based partitions. + * @throws Exception + */ + @Test + public void testDatePartitionedTable() throws Exception { + String instant = "100"; + hoodieTestTable = hoodieTestTable.addCommit(instant); + // Generate 10 files under each partition + DATE_PARTITIONS.stream().forEach(p -> { + try { + hoodieTestTable = hoodieTestTable.withBaseFilesInPartition(p, IntStream.range(0, 10).toArray()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, true); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 3); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(basePath + "/" + DATE_PARTITIONS.get(0))).length == 10); + } + + /** + * Test listing of partitions result for date based partitions with assumeDataPartitioning = false. + * @throws Exception + */ + @Test + public void testDatePartitionedTableWithAssumeDateIsFalse() throws Exception { + String instant = "100"; + hoodieTestTable = hoodieTestTable.addCommit(instant); + // Generate 10 files under each partition + DATE_PARTITIONS.stream().forEach(p -> { + try { + hoodieTestTable = hoodieTestTable.withBaseFilesInPartition(p, IntStream.range(0, 10).toArray()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, false); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 0); + } + + @Test + public void testOneLevelPartitionedTable() throws Exception { + String instant = "100"; + hoodieTestTable = hoodieTestTable.addCommit(instant); + // Generate 10 files under each partition + ONE_LEVEL_PARTITIONS.stream().forEach(p -> { + try { + hoodieTestTable = hoodieTestTable.withPartitionMetaFiles(p) + .withBaseFilesInPartition(p, IntStream.range(0, 10).toArray()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, false); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 3); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(basePath + "/" + ONE_LEVEL_PARTITIONS.get(0))).length == 10); + } + + @Test + public void testMultiLevelPartitionedTable() throws Exception { + String instant = "100"; + hoodieTestTable = hoodieTestTable.addCommit(instant); + // Generate 10 files under each partition + MULTI_LEVEL_PARTITIONS.stream().forEach(p -> { + try { + hoodieTestTable = hoodieTestTable.withPartitionMetaFiles(p) + .withBaseFilesInPartition(p, IntStream.range(0, 10).toArray()); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, false); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 3); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(basePath + "/" + MULTI_LEVEL_PARTITIONS.get(0))).length == 10); + } + + @Test + public void testMultiLevelEmptyPartitionTable() throws Exception { + String instant = "100"; + hoodieTestTable = hoodieTestTable.addCommit(instant); + // Generate 10 files under each partition + MULTI_LEVEL_PARTITIONS.stream().forEach(p -> { + try { + hoodieTestTable = hoodieTestTable.withPartitionMetaFiles(p); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + new FileSystemBackedTableMetadata(localEngineContext, new SerializableConfiguration(metaClient.getHadoopConf()), basePath, false); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllPartitionPaths().size() == 3); + Assertions.assertTrue(fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(basePath + "/" + MULTI_LEVEL_PARTITIONS.get(0))).length == 0); + } + +} diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index ba132904e19f1..1c356b098599d 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 @@ -133,6 +133,12 @@ ${project.version} + + org.apache.hudi + hudi-java-client + ${project.version} + + org.apache.hudi hudi-spark-client diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java b/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java index 4a9868bd39fea..71c6408ccb2cd 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/common/HoodieExampleDataGenerator.java @@ -55,7 +55,7 @@ public class HoodieExampleDataGenerator> { public static final String[] DEFAULT_PARTITION_PATHS = {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH}; public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"triprec\",\"fields\": [ " - + "{\"name\": \"ts\",\"type\": \"double\"},{\"name\": \"uuid\", \"type\": \"string\"}," + + "{\"name\": \"ts\",\"type\": \"long\"},{\"name\": \"uuid\", \"type\": \"string\"}," + "{\"name\": \"rider\", \"type\": \"string\"},{\"name\": \"driver\", \"type\": \"string\"}," + "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"}," + "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"}," diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java new file mode 100644 index 0000000000000..6cb1ea9d2fda5 --- /dev/null +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieJavaWriteClientExample.java @@ -0,0 +1,119 @@ +/* + * 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.examples.java; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.examples.common.HoodieExampleDataGenerator; +import org.apache.hudi.index.HoodieIndex; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + + +/** + * Simple examples of #{@link HoodieJavaWriteClient}. + * + * Usage: HoodieWriteClientExample + * and describe root path of hudi and table name + * for example, `HoodieWriteClientExample file:///tmp/hoodie/sample-table hoodie_rt` + */ +public class HoodieJavaWriteClientExample { + + private static final Logger LOG = LogManager.getLogger(HoodieJavaWriteClientExample.class); + + private static String tableType = HoodieTableType.COPY_ON_WRITE.name(); + + public static void main(String[] args) throws Exception { + if (args.length < 2) { + System.err.println("Usage: HoodieWriteClientExample "); + System.exit(1); + } + String tablePath = args[0]; + String tableName = args[1]; + + // Generator of some records to be loaded in. + HoodieExampleDataGenerator dataGen = new HoodieExampleDataGenerator<>(); + + Configuration hadoopConf = new Configuration(); + // initialize the table, if not done already + Path path = new Path(tablePath); + FileSystem fs = FSUtils.getFs(tablePath, hadoopConf); + if (!fs.exists(path)) { + HoodieTableMetaClient.initTableType(hadoopConf, tablePath, HoodieTableType.valueOf(tableType), + tableName, HoodieAvroPayload.class.getName()); + } + + // Create the write client to write some records in + HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .withDeleteParallelism(2).forTable(tableName) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build(); + HoodieJavaWriteClient client = + new HoodieJavaWriteClient<>(new HoodieJavaEngineContext(hadoopConf), cfg); + + // inserts + String newCommitTime = client.startCommit(); + LOG.info("Starting commit " + newCommitTime); + + List> records = dataGen.generateInserts(newCommitTime, 10); + List> recordsSoFar = new ArrayList<>(records); + List> writeRecords = + recordsSoFar.stream().map(r -> new HoodieRecord(r)).collect(Collectors.toList()); + client.upsert(writeRecords, newCommitTime); + + // updates + newCommitTime = client.startCommit(); + LOG.info("Starting commit " + newCommitTime); + List> toBeUpdated = dataGen.generateUpdates(newCommitTime, 2); + records.addAll(toBeUpdated); + recordsSoFar.addAll(toBeUpdated); + writeRecords = + recordsSoFar.stream().map(r -> new HoodieRecord(r)).collect(Collectors.toList()); + client.upsert(writeRecords, newCommitTime); + + // Delete + newCommitTime = client.startCommit(); + LOG.info("Starting commit " + newCommitTime); + // just delete half of the records + int numToDelete = recordsSoFar.size() / 2; + List toBeDeleted = + recordsSoFar.stream().map(HoodieRecord::getKey).limit(numToDelete).collect(Collectors.toList()); + client.delete(toBeDeleted, newCommitTime); + + client.close(); + } +} diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml index 4b8cfd78e1009..3d100c7728424 100644 --- a/hudi-flink/pom.xml +++ b/hudi-flink/pom.xml @@ -17,12 +17,11 @@ limitations under the License. --> - + hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 @@ -173,5 +172,102 @@ bijection-avro_${scala.binary.version} 0.9.7 + + + + org.junit.jupiter + junit-jupiter-api + test + + + org.junit.jupiter + junit-jupiter-engine + test + + + org.junit.vintage + junit-vintage-engine + test + + + org.junit.jupiter + junit-jupiter-params + test + + + org.mockito + mockito-junit-jupiter + test + + + org.junit.platform + junit-platform-runner + test + + + org.junit.platform + junit-platform-suite-api + test + + + org.junit.platform + junit-platform-commons + test + + + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-flink-client + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-hadoop-mr + ${project.version} + test + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + test + tests + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + test + tests + + diff --git a/hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java index 0c9991da3db4a..b65b43c0d7a26 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java +++ b/hudi-flink/src/main/java/org/apache/hudi/HoodieFlinkStreamer.java @@ -19,9 +19,11 @@ package org.apache.hudi; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.operator.InstantGenerateOperator; import org.apache.hudi.operator.KeyedWriteProcessFunction; import org.apache.hudi.operator.KeyedWriteProcessOperator; @@ -42,11 +44,11 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; import java.util.ArrayList; import java.util.List; import java.util.Objects; -import java.util.Properties; /** * An Utility which can incrementally consume data from Kafka and apply it to the target table. @@ -73,21 +75,28 @@ public static void main(String[] args) throws Exception { env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); } - Properties kafkaProps = StreamerUtil.getKafkaProps(cfg); + TypedProperties props = StreamerUtil.getProps(cfg); + + // add kafka config + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers); + props.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId); + + // add data source config + props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, cfg.payloadClassName); + props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, cfg.sourceOrderingField); // Read from kafka source DataStream inputRecords = - env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), kafkaProps)) + env.addSource(new FlinkKafkaConsumer<>(cfg.kafkaTopic, new SimpleStringSchema(), props)) .filter(Objects::nonNull) - .map(new JsonStringToHoodieRecordMapFunction(cfg)) + .map(new JsonStringToHoodieRecordMapFunction(props)) .name("kafka_to_hudi_record") .uid("kafka_to_hudi_record_uid"); - // InstantGenerateOperator helps to emit globally unique instantTime, it must be executed in one parallelism + // InstantGenerateOperator helps to emit globally unique instantTime inputRecords.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator()) .name("instant_generator") .uid("instant_generator_id") - .setParallelism(1) // Keyby partition path, to avoid multiple subtasks writing to a partition at the same time .keyBy(HoodieRecord::getPartitionPath) diff --git a/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java index 165eeb087b207..75c766837c0c6 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/operator/InstantGenerateOperator.java @@ -22,8 +22,8 @@ import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.common.HoodieFlinkEngineContext; -import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -38,10 +38,13 @@ import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.PathFilter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,15 +52,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** - * Operator helps to generate globally unique instant, it must be executed in one parallelism. Before generate a new - * instant , {@link InstantGenerateOperator} will always check whether the last instant has completed. if it is - * completed, a new instant will be generated immediately, otherwise, wait and check the state of last instant until - * time out and throw an exception. + * Operator helps to generate globally unique instant. Before generate a new instant {@link InstantGenerateOperator} + * will always check whether the last instant has completed. if it is completed and has records flows in, a new instant + * will be generated immediately, otherwise, wait and check the state of last instant until time out and throw an exception. */ public class InstantGenerateOperator extends AbstractStreamOperator implements OneInputStreamOperator { @@ -71,16 +73,20 @@ public class InstantGenerateOperator extends AbstractStreamOperator latestInstantList = new ArrayList<>(1); private transient ListState latestInstantState; - private List bufferedRecords = new LinkedList(); - private transient ListState recordsState; private Integer retryTimes; private Integer retryInterval; + private static final String DELIMITER = "_"; + private static final String INSTANT_MARKER_FOLDER_NAME = ".instant_marker"; + private transient boolean isMain = false; + private transient AtomicLong recordCounter = new AtomicLong(0); + private StreamingRuntimeContext runtimeContext; + private int indexOfThisSubtask; @Override public void processElement(StreamRecord streamRecord) throws Exception { if (streamRecord.getValue() != null) { - bufferedRecords.add(streamRecord); output.collect(streamRecord); + recordCounter.incrementAndGet(); } } @@ -88,7 +94,7 @@ public void processElement(StreamRecord streamRecord) throws Excep public void open() throws Exception { super.open(); // get configs from runtimeContext - cfg = (HoodieFlinkStreamer.Config) getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + cfg = (HoodieFlinkStreamer.Config) runtimeContext.getExecutionConfig().getGlobalJobParameters(); // retry times retryTimes = Integer.valueOf(cfg.blockRetryTime); @@ -102,65 +108,78 @@ public void open() throws Exception { // Hadoop FileSystem fs = FSUtils.getFs(cfg.targetBasePath, serializableHadoopConf.get()); - TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); + if (isMain) { + TaskContextSupplier taskContextSupplier = new FlinkTaskContextSupplier(null); - // writeClient - writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg), true); + // writeClient + writeClient = new HoodieFlinkWriteClient(new HoodieFlinkEngineContext(taskContextSupplier), StreamerUtil.getHoodieClientConfig(cfg), true); - // init table, create it if not exists. - initTable(); + // init table, create it if not exists. + initTable(); + + // create instant marker directory + createInstantMarkerDir(); + } } @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { super.prepareSnapshotPreBarrier(checkpointId); - // check whether the last instant is completed, if not, wait 10s and then throws an exception - if (!StringUtils.isNullOrEmpty(latestInstant)) { - doCheck(); - // last instant completed, set it empty - latestInstant = ""; - } - - // no data no new instant - if (!bufferedRecords.isEmpty()) { - latestInstant = startNewInstant(checkpointId); + String instantMarkerFileName = String.format("%d%s%d%s%d", indexOfThisSubtask, DELIMITER, checkpointId, DELIMITER, recordCounter.get()); + Path path = new Path(new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, INSTANT_MARKER_FOLDER_NAME), instantMarkerFileName); + // create marker file + fs.create(path, true); + LOG.info("Subtask [{}] at checkpoint [{}] created marker file [{}]", indexOfThisSubtask, checkpointId, instantMarkerFileName); + if (isMain) { + // check whether the last instant is completed, will try specific times until an exception is thrown + if (!StringUtils.isNullOrEmpty(latestInstant)) { + doCheck(); + // last instant completed, set it empty + latestInstant = ""; + } + boolean receivedDataInCurrentCP = checkReceivedData(checkpointId); + // no data no new instant + if (receivedDataInCurrentCP) { + latestInstant = startNewInstant(checkpointId); + } } } @Override public void initializeState(StateInitializationContext context) throws Exception { - // instantState - ListStateDescriptor latestInstantStateDescriptor = new ListStateDescriptor("latestInstant", String.class); - latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor); - - // recordState - ListStateDescriptor recordsStateDescriptor = new ListStateDescriptor("recordsState", StreamRecord.class); - recordsState = context.getOperatorStateStore().getListState(recordsStateDescriptor); - - if (context.isRestored()) { - Iterator latestInstantIterator = latestInstantState.get().iterator(); - latestInstantIterator.forEachRemaining(x -> latestInstant = x); - LOG.info("InstantGenerateOperator initializeState get latestInstant [{}]", latestInstant); - - Iterator recordIterator = recordsState.get().iterator(); - bufferedRecords.clear(); - recordIterator.forEachRemaining(x -> bufferedRecords.add(x)); + runtimeContext = getRuntimeContext(); + indexOfThisSubtask = runtimeContext.getIndexOfThisSubtask(); + isMain = indexOfThisSubtask == 0; + + if (isMain) { + // instantState + ListStateDescriptor latestInstantStateDescriptor = new ListStateDescriptor<>("latestInstant", String.class); + latestInstantState = context.getOperatorStateStore().getListState(latestInstantStateDescriptor); + + if (context.isRestored()) { + Iterator latestInstantIterator = latestInstantState.get().iterator(); + latestInstantIterator.forEachRemaining(x -> latestInstant = x); + LOG.info("Restoring the latest instant [{}] from the state", latestInstant); + } } } @Override public void snapshotState(StateSnapshotContext functionSnapshotContext) throws Exception { - if (latestInstantList.isEmpty()) { - latestInstantList.add(latestInstant); + long checkpointId = functionSnapshotContext.getCheckpointId(); + long recordSize = recordCounter.get(); + if (isMain) { + LOG.info("Update latest instant [{}] records size [{}] checkpointId [{}]", latestInstant, recordSize, checkpointId); + if (latestInstantList.isEmpty()) { + latestInstantList.add(latestInstant); + } else { + latestInstantList.set(0, latestInstant); + } + latestInstantState.update(latestInstantList); } else { - latestInstantList.set(0, latestInstant); + LOG.info("Task instance {} received {} records in checkpoint [{}]", indexOfThisSubtask, recordSize, checkpointId); } - latestInstantState.update(latestInstantList); - LOG.info("Update latest instant [{}]", latestInstant); - - recordsState.update(bufferedRecords); - LOG.info("Update records state size = [{}]", bufferedRecords.size()); - bufferedRecords.clear(); + recordCounter.set(0); } /** @@ -185,10 +204,10 @@ private void doCheck() throws InterruptedException { int tryTimes = 0; while (tryTimes < retryTimes) { tryTimes++; - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); if (rollbackPendingCommits.contains(latestInstant)) { rollbackPendingCommits.forEach(x -> sb.append(x).append(",")); - LOG.warn("Latest transaction [{}] is not completed! unCompleted transaction:[{}],try times [{}]", latestInstant, sb.toString(), tryTimes); + LOG.warn("Latest transaction [{}] is not completed! unCompleted transaction:[{}],try times [{}]", latestInstant, sb, tryTimes); TimeUnit.SECONDS.sleep(retryInterval); rollbackPendingCommits = writeClient.getInflightsAndRequestedInstants(commitType); } else { @@ -196,7 +215,7 @@ private void doCheck() throws InterruptedException { return; } } - throw new InterruptedException("Last instant costs more than ten second, stop task now"); + throw new InterruptedException(String.format("Last instant costs more than %s second, stop task now", retryTimes * retryInterval)); } @@ -222,4 +241,60 @@ public void close() throws Exception { fs.close(); } } + + private boolean checkReceivedData(long checkpointId) throws InterruptedException, IOException { + int numberOfParallelSubtasks = runtimeContext.getNumberOfParallelSubtasks(); + FileStatus[] fileStatuses; + Path instantMarkerPath = new Path(HoodieTableMetaClient.AUXILIARYFOLDER_NAME, INSTANT_MARKER_FOLDER_NAME); + // waiting all subtask create marker file ready + while (true) { + Thread.sleep(500L); + fileStatuses = fs.listStatus(instantMarkerPath, new PathFilter() { + @Override + public boolean accept(Path pathname) { + return pathname.getName().contains(String.format("%s%d%s", DELIMITER, checkpointId, DELIMITER)); + } + }); + + // is ready + if (fileStatuses != null && fileStatuses.length == numberOfParallelSubtasks) { + break; + } + } + + boolean receivedData = false; + // check whether has data in this checkpoint and delete maker file. + for (FileStatus fileStatus : fileStatuses) { + Path path = fileStatus.getPath(); + String name = path.getName(); + // has data + if (Long.parseLong(name.split(DELIMITER)[2]) > 0) { + receivedData = true; + break; + } + } + + // delete all marker file + cleanMarkerDir(instantMarkerPath); + + return receivedData; + } + + private void createInstantMarkerDir() throws IOException { + // Always create instantMarkerFolder which is needed for InstantGenerateOperator + final Path instantMarkerFolder = new Path(new Path(cfg.targetBasePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME), INSTANT_MARKER_FOLDER_NAME); + if (!fs.exists(instantMarkerFolder)) { + fs.mkdirs(instantMarkerFolder); + } else { + // Clean marker dir. + cleanMarkerDir(instantMarkerFolder); + } + } + + private void cleanMarkerDir(Path instantMarkerFolder) throws IOException { + FileStatus[] fileStatuses = fs.listStatus(instantMarkerFolder); + for (FileStatus fileStatus : fileStatuses) { + fs.delete(fileStatus.getPath(), true); + } + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java b/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java index f8784841374f9..4eda371b4a017 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/source/JsonStringToHoodieRecordMapFunction.java @@ -18,22 +18,24 @@ package org.apache.hudi.source; -import org.apache.hudi.HoodieFlinkStreamer; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieFlinkStreamerException; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.util.Option; import org.apache.hudi.keygen.KeyGenerator; +import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.schema.FilebasedSchemaProvider; import org.apache.hudi.util.AvroConvertor; import org.apache.hudi.util.StreamerUtil; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.functions.MapFunction; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; /** @@ -41,34 +43,46 @@ */ public class JsonStringToHoodieRecordMapFunction implements MapFunction { - private static Logger LOG = LoggerFactory.getLogger(JsonStringToHoodieRecordMapFunction.class); - - private final HoodieFlinkStreamer.Config cfg; private TypedProperties props; private KeyGenerator keyGenerator; private AvroConvertor avroConvertor; + private Option schemaStr; + private String payloadClassName; + private String orderingField; - public JsonStringToHoodieRecordMapFunction(HoodieFlinkStreamer.Config cfg) { - this.cfg = cfg; + public JsonStringToHoodieRecordMapFunction(TypedProperties props) { + this(props, Option.empty()); + } + + public JsonStringToHoodieRecordMapFunction(TypedProperties props, Option schemaStr) { + this.props = props; + this.schemaStr = schemaStr; init(); } @Override public HoodieRecord map(String value) throws Exception { - GenericRecord gr = avroConvertor.fromJson(value); - HoodieRecordPayload payload = StreamerUtil.createPayload(cfg.payloadClassName, gr, - (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false)); + GenericRecord gr = this.avroConvertor.fromJson(value); + HoodieRecordPayload payload = StreamerUtil.createPayload(this.payloadClassName, gr, + (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, this.orderingField, false)); - return new HoodieRecord<>(keyGenerator.getKey(gr), payload); + return new HoodieRecord<>(this.keyGenerator.getKey(gr), payload); } private void init() { - this.props = StreamerUtil.getProps(cfg); - avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema()); + if (schemaStr.isPresent()) { + this.avroConvertor = new AvroConvertor(new Schema.Parser().parse(schemaStr.get())); + } else { + this.avroConvertor = new AvroConvertor(new FilebasedSchemaProvider(props).getSourceSchema()); + } + this.payloadClassName = props.getString(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, + OverwriteWithLatestAvroPayload.class.getName()); + this.orderingField = props.getString(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "ts"); try { - keyGenerator = StreamerUtil.createKeyGenerator(props); + this.keyGenerator = StreamerUtil.createKeyGenerator(props); } catch (IOException e) { - LOG.error("Init keyGenerator failed ", e); + throw new HoodieFlinkStreamerException(String.format("KeyGenerator %s initialization failed", + props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, SimpleAvroKeyGenerator.class.getName())), e); } } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index db7ad253aeadf..71de651b28a5e 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -22,14 +22,14 @@ import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieNotSupportedException; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.schema.FilebasedSchemaProvider; @@ -38,7 +38,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,19 +45,11 @@ import java.io.IOException; import java.io.StringReader; import java.util.List; -import java.util.Properties; public class StreamerUtil { private static Logger LOG = LoggerFactory.getLogger(StreamerUtil.class); - public static Properties getKafkaProps(HoodieFlinkStreamer.Config cfg) { - Properties result = new Properties(); - result.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cfg.kafkaBootstrapServers); - result.put(ConsumerConfig.GROUP_ID_CONFIG, cfg.kafkaGroupId); - return result; - } - public static TypedProperties getProps(HoodieFlinkStreamer.Config cfg) { return readConfig( FSUtils.getFs(cfg.propsFilePath, getHadoopConf()), @@ -134,10 +125,11 @@ public static HoodieRecordPayload createPayload(String payloadClass, GenericReco public static HoodieWriteConfig getHoodieClientConfig(HoodieFlinkStreamer.Config cfg) { FileSystem fs = FSUtils.getFs(cfg.targetBasePath, getHadoopConf()); HoodieWriteConfig.Builder builder = - HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, true) + HoodieWriteConfig.newBuilder().withEngineType(EngineType.FLINK).withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, true) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) + .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) + .build()) .forTable(cfg.targetTableName) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()) .withAutoCommit(false) .withProps(readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) .getConfig()); diff --git a/hudi-flink/src/test/java/org/apache/hudi/source/TestJsonStringToHoodieRecordMapFunction.java b/hudi-flink/src/test/java/org/apache/hudi/source/TestJsonStringToHoodieRecordMapFunction.java new file mode 100644 index 0000000000000..98066e96aea30 --- /dev/null +++ b/hudi-flink/src/test/java/org/apache/hudi/source/TestJsonStringToHoodieRecordMapFunction.java @@ -0,0 +1,91 @@ +/* + * 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.source; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.testutils.HoodieFlinkClientTestHarness; + +import org.apache.avro.Schema; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; + +public class TestJsonStringToHoodieRecordMapFunction extends HoodieFlinkClientTestHarness { + @BeforeEach + public void init() { + initPath(); + initTestDataGenerator(); + initFileSystem(); + initFlinkMiniCluster(); + } + + @AfterEach + public void clean() throws Exception { + cleanupTestDataGenerator(); + cleanupFileSystem(); + cleanupFlinkMiniCluster(); + } + + @Test + @Disabled + public void testMapFunction() throws Exception { + final String newCommitTime = "001"; + final int numRecords = 10; + List records = dataGen.generateInserts(newCommitTime, numRecords); + List recordStr = RawTripTestPayload.recordsToStrings(records); + Schema schema = AVRO_SCHEMA; + + TypedProperties props = new TypedProperties(); + props.put(HoodieWriteConfig.WRITE_PAYLOAD_CLASS, OverwriteWithLatestAvroPayload.class.getName()); + props.put(HoodieWriteConfig.PRECOMBINE_FIELD_PROP, "timestamp"); + props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key"); + props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionPath"); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + SimpleTestSinkFunction.valuesList.clear(); + env.fromCollection(recordStr) + .map(new JsonStringToHoodieRecordMapFunction(props, Option.of(schema.toString()))) + .addSink(new SimpleTestSinkFunction()); + env.execute(); + + // input records all present in the sink + Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.size()); + + // input keys all present in the sink + Set inputKeySet = records.stream().map(r -> r.getKey().getRecordKey()).collect(Collectors.toSet()); + Assertions.assertEquals(10, SimpleTestSinkFunction.valuesList.stream() + .map(r -> inputKeySet.contains(r.getRecordKey())).filter(b -> b).count()); + } +} diff --git a/hudi-flink/src/test/resources/log4j-surefire-quiet.properties b/hudi-flink/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..2b94ea2903067 --- /dev/null +++ b/hudi-flink/src/test/resources/log4j-surefire-quiet.properties @@ -0,0 +1,30 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# CONSOLE is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# CONSOLE uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL \ No newline at end of file diff --git a/hudi-flink/src/test/resources/log4j-surefire.properties b/hudi-flink/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000000000..32af462093ae5 --- /dev/null +++ b/hudi-flink/src/test/resources/log4j-surefire.properties @@ -0,0 +1,31 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache=INFO +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# A1 is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml index ce2c03fd948e6..3c0f2ea6632e4 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java index 1747888b15ad7..e3bac0b4e8f9a 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java @@ -31,7 +31,6 @@ import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapreduce.Job; -import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -97,20 +96,9 @@ public FileStatus[] listStatus(JobConf job) throws IOException { // process snapshot queries next. List snapshotPaths = inputPathHandler.getSnapshotPaths(); if (snapshotPaths.size() > 0) { - setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()])); - FileStatus[] fileStatuses = super.listStatus(job); - Map> groupedFileStatus = - HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, HoodieFileFormat.HFILE.getFileExtension(), - tableMetaClientMap.values()); - LOG.info("Found a total of " + groupedFileStatus.size() + " groups"); - for (Map.Entry> entry : groupedFileStatus.entrySet()) { - List result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue()); - if (result != null) { - returns.addAll(result); - } - } + returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths)); } - return returns.toArray(new FileStatus[returns.size()]); + return returns.toArray(new FileStatus[0]); } /** diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 8b89949e803b6..0288cbd4ebc26 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -18,7 +18,6 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; @@ -109,20 +108,9 @@ public FileStatus[] listStatus(JobConf job) throws IOException { // process snapshot queries next. List snapshotPaths = inputPathHandler.getSnapshotPaths(); if (snapshotPaths.size() > 0) { - setInputPaths(job, snapshotPaths.toArray(new Path[snapshotPaths.size()])); - FileStatus[] fileStatuses = super.listStatus(job); - Map> groupedFileStatus = - HoodieInputFormatUtils.groupFileStatusForSnapshotPaths(fileStatuses, - HoodieFileFormat.PARQUET.getFileExtension(), tableMetaClientMap.values()); - LOG.info("Found a total of " + groupedFileStatus.size() + " groups"); - for (Map.Entry> entry : groupedFileStatus.entrySet()) { - List result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue()); - if (result != null) { - returns.addAll(result); - } - } + returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths)); } - return returns.toArray(new FileStatus[returns.size()]); + return returns.toArray(new FileStatus[0]); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java index 1e616f896bb30..da45fa6380f2d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java @@ -18,18 +18,20 @@ package org.apache.hudi.hadoop; -import java.util.Map; -import java.util.Set; -import org.apache.hadoop.conf.Configurable; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -41,6 +43,8 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; /** @@ -79,6 +83,9 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial */ private SerializableConfiguration conf; + private transient HoodieLocalEngineContext engineContext; + + private transient FileSystem fs; public HoodieROTablePathFilter() { @@ -108,6 +115,10 @@ private Path safeGetParentsParent(Path path) { @Override public boolean accept(Path path) { + if (engineContext == null) { + this.engineContext = new HoodieLocalEngineContext(this.conf.get()); + } + if (LOG.isDebugEnabled()) { LOG.debug("Checking acceptance for path " + path); } @@ -156,6 +167,7 @@ public boolean accept(Path path) { } if (baseDir != null) { + HoodieTableFileSystemView fsView = null; try { HoodieTableMetaClient metaClient = metaClientCache.get(baseDir.toString()); if (null == metaClient) { @@ -163,9 +175,10 @@ public boolean accept(Path path) { metaClientCache.put(baseDir.toString(), metaClient); } - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder)); - List latestFiles = fsView.getLatestBaseFiles().collect(Collectors.toList()); + fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf())); + String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder); + List latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList()); // populate the cache if (!hoodiePathCache.containsKey(folder.toString())) { hoodiePathCache.put(folder.toString(), new HashSet<>()); @@ -189,6 +202,10 @@ public boolean accept(Path path) { } nonHoodiePathCache.add(folder.toString()); return true; + } finally { + if (fsView != null) { + fsView.close(); + } } } else { // files is at < 3 level depth in FS tree, can't be hoodie dataset diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index b710b599b0ba8..a98a230102971 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -63,17 +63,18 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept // NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit // but can return records for completed commits > the commit we are trying to read (if using // readCommit() API) - return new HoodieMergedLogRecordScanner( - FSUtils.getFs(split.getPath().toString(), jobConf), - split.getBasePath(), - split.getDeltaLogPaths(), - usesCustomPayload ? getWriterSchema() : getReaderSchema(), - split.getMaxCommitTime(), - HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf), - Boolean.parseBoolean(jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), - false, - jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), - jobConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)); + return HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(FSUtils.getFs(split.getPath().toString(), jobConf)) + .withBasePath(split.getBasePath()) + .withLogFilePaths(split.getDeltaLogPaths()) + .withReaderSchema(usesCustomPayload ? getWriterSchema() : getReaderSchema()) + .withLatestInstantTime(split.getMaxCommitTime()) + .withMaxMemorySizeInBytes(HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(jobConf)) + .withReadBlocksLazily(Boolean.parseBoolean(jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) + .withReverseReader(false) + .withBufferSize(jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withSpillableMapBasePath(jobConf.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + .build(); } @Override @@ -108,7 +109,7 @@ public boolean next(NullWritable aVoid, ArrayWritable arrayWritable) throws IOEx if (usesCustomPayload) { // If using a custom payload, return only the projection fields. The readerSchema is a schema derived from // the writerSchema with only the projection fields - recordToReturn = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(rec.get(), getReaderSchema()); + recordToReturn = HoodieAvroUtils.rewriteRecord(rec.get(), getReaderSchema()); } // we assume, a later safe record in the log, is newer than what we have in the map & // replace it. Since we want to return an arrayWritable which is the same length as the elements in the latest diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java index 76de84bd9a52e..d209a5a388fe6 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -77,15 +77,22 @@ public RealtimeUnmergedRecordReader(RealtimeSplit split, JobConf job, Option.empty(), x -> x, new DefaultSizeEstimator<>()); // Consumer of this record reader this.iterator = this.executor.getQueue().iterator(); - this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), this.jobConf), - split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(), - Boolean.parseBoolean(this.jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)), - false, this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> { + this.logRecordScanner = HoodieUnMergedLogRecordScanner.newBuilder() + .withFileSystem(FSUtils.getFs(split.getPath().toString(), this.jobConf)) + .withBasePath(split.getBasePath()) + .withLogFilePaths(split.getDeltaLogPaths()) + .withReaderSchema(getReaderSchema()) + .withLatestInstantTime(split.getMaxCommitTime()) + .withReadBlocksLazily(Boolean.parseBoolean(this.jobConf.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))) + .withReverseReader(false) + .withBufferSize(this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withLogRecordScannerCallback(record -> { // convert Hoodie log record to Hadoop AvroWritable and buffer GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get(); ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema()); this.executor.getQueue().insertRecord(aWritable); - }); + }) + .build(); // Start reading and buffering this.executor.startProducers(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index b36885106cce0..9f9813654cc91 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -18,15 +18,19 @@ package org.apache.hudi.hadoop.utils; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; @@ -38,6 +42,7 @@ import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -62,6 +67,11 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; +import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP; +import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP; + public class HoodieInputFormatUtils { // These positions have to be deterministic across all tables @@ -236,7 +246,7 @@ public static Option getAffectedPartitions(List commitsTo return false; }) .collect(Collectors.joining(",")); - return Option.of(incrementalInputPaths); + return StringUtils.isNullOrEmpty(incrementalInputPaths) ? Option.empty() : Option.of(incrementalInputPaths); } /** @@ -391,32 +401,64 @@ public static Map> groupFileStatusForSna return grouped; } - /** - * Filters data files for a snapshot queried table. - * @param job - * @param metadata - * @param fileStatuses - * @return - */ - public static List filterFileStatusForSnapshotMode( - JobConf job, HoodieTableMetaClient metadata, List fileStatuses) throws IOException { - FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]); - if (LOG.isDebugEnabled()) { - LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata); + public static Map> groupSnapshotPathsByMetaClient( + Collection metaClientList, + List snapshotPaths + ) { + Map> grouped = new HashMap<>(); + metaClientList.forEach(metaClient -> grouped.put(metaClient, new ArrayList<>())); + for (Path path : snapshotPaths) { + // Find meta client associated with the input path + metaClientList.stream().filter(metaClient -> path.toString().contains(metaClient.getBasePath())) + .forEach(metaClient -> grouped.get(metaClient).add(path)); } - // Get all commits, delta commits, compactions, as all of them produce a base parquet file today - HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - TableFileSystemView.BaseFileOnlyView roView = new HoodieTableFileSystemView(metadata, timeline, statuses); - // filter files on the latest commit found - List filteredFiles = roView.getLatestBaseFiles().collect(Collectors.toList()); - LOG.info("Total paths to process after hoodie filter " + filteredFiles.size()); + return grouped; + } + + public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) { + return HoodieMetadataConfig.newBuilder() + .enable(conf.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS)) + .validate(conf.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE)) + .build(); + } + + public static List filterFileStatusForSnapshotMode(JobConf job, Map tableMetaClientMap, + List snapshotPaths) throws IOException { + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job); List returns = new ArrayList<>(); - for (HoodieBaseFile filteredFile : filteredFiles) { - if (LOG.isDebugEnabled()) { - LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); + + Map> groupedPaths = HoodieInputFormatUtils + .groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths); + Map fsViewCache = new HashMap<>(); + LOG.info("Found a total of " + groupedPaths.size() + " groups"); + + try { + for (Map.Entry> entry : groupedPaths.entrySet()) { + HoodieTableMetaClient metaClient = entry.getKey(); + if (LOG.isDebugEnabled()) { + LOG.debug("Hoodie Metadata initialized with completed commit instant as :" + metaClient); + } + + HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient -> + FileSystemViewManager.createInMemoryFileSystemView(engineContext, tableMetaClient, buildMetadataConfig(job))); + List filteredBaseFiles = new ArrayList<>(); + for (Path p : entry.getValue()) { + String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p); + List matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList()); + filteredBaseFiles.addAll(matched); + } + + LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size()); + for (HoodieBaseFile filteredFile : filteredBaseFiles) { + if (LOG.isDebugEnabled()) { + LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); + } + filteredFile = refreshFileStatus(job, filteredFile); + returns.add(getFileStatus(filteredFile)); + } } - filteredFile = refreshFileStatus(job, filteredFile); - returns.add(getFileStatus(filteredFile)); + } finally { + fsViewCache.forEach(((metaClient, fsView) -> fsView.close())); } return returns; } @@ -443,4 +485,44 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi } } + /** + * Iterate through a list of commits in ascending order, and extract the file status of + * all affected files from the commits metadata grouping by partition path. If the files has + * been touched multiple times in the given commits, the return value will keep the one + * from the latest commit. + * @param basePath + * @param commitsToCheck + * @param timeline + * @return HashMap> + * @throws IOException + */ + public static HashMap> listAffectedFilesForCommits( + Path basePath, List commitsToCheck, HoodieTimeline timeline) throws IOException { + // TODO: Use HoodieMetaTable to extract affected file directly. + HashMap> partitionToFileStatusesMap = new HashMap<>(); + List sortedCommitsToCheck = new ArrayList<>(commitsToCheck); + sortedCommitsToCheck.sort(HoodieInstant::compareTo); + // Iterate through the given commits. + for (HoodieInstant commit: sortedCommitsToCheck) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(), + HoodieCommitMetadata.class); + // Iterate through all the affected partitions of a commit. + for (Map.Entry> entry: commitMetadata.getPartitionToWriteStats().entrySet()) { + if (!partitionToFileStatusesMap.containsKey(entry.getKey())) { + partitionToFileStatusesMap.put(entry.getKey(), new HashMap<>()); + } + // Iterate through all the written files of this partition. + for (HoodieWriteStat stat : entry.getValue()) { + String relativeFilePath = stat.getPath(); + Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null; + if (fullPath != null) { + FileStatus fs = new FileStatus(stat.getFileSizeInBytes(), false, 0, 0, + 0, fullPath); + partitionToFileStatusesMap.get(entry.getKey()).put(fullPath.getName(), fs); + } + } + } + } + return partitionToFileStatusesMap; + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 760dd961be65b..ce770bad15b78 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.utils; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -26,6 +27,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; @@ -63,16 +65,25 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream partitionsToMetaClient = getTableMetaClientByBasePath(conf, partitionsToParquetSplits.keySet()); + // Create file system cache so metadata table is only instantiated once. Also can benefit normal file listing if + // partition path is listed twice so file groups will already be loaded in file system + Map fsCache = new HashMap<>(); // for all unique split parents, obtain all delta files based on delta commit timeline, // grouped on file id List rtSplits = new ArrayList<>(); - partitionsToParquetSplits.keySet().forEach(partitionPath -> { - // for each partition path obtain the data & log file groupings, then map back to inputsplits - HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); - HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline()); - String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); - - try { + try { + partitionsToParquetSplits.keySet().forEach(partitionPath -> { + // for each partition path obtain the data & log file groupings, then map back to inputsplits + HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); + if (!fsCache.containsKey(metaClient)) { + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf); + HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext, + metaClient, HoodieInputFormatUtils.buildMetadataConfig(conf)); + fsCache.put(metaClient, fsView); + } + HoodieTableFileSystemView fsView = fsCache.get(metaClient); + + String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); // Both commit and delta-commits are included - pick the latest completed one Option latestCompletedInstant = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); @@ -86,7 +97,7 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream FSUtils.getFileId(split.getPath().getName()))); // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) + HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)) .filterCompletedInstants().lastInstant().get().getTimestamp(); latestFileSlices.forEach(fileSlice -> { List dataFileSplits = groupedInputSplits.get(fileSlice.getFileId()); @@ -102,7 +113,7 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, Stream view.close()); + } LOG.info("Returning a total splits of " + rtSplits.size()); return rtSplits.toArray(new InputSplit[0]); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java index 27838897d5ec3..59214ac445049 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java @@ -117,7 +117,6 @@ public void testPendingCompactionWithActiveCommits() throws IOException { assertFalse(filteredTimeline.containsInstant(t5)); assertFalse(filteredTimeline.containsInstant(t6)); - // remove compaction instant and setup timeline again instants.remove(t3); timeline = new HoodieActiveTimeline(metaClient); @@ -239,6 +238,33 @@ public void testIncrementalSimple() throws IOException { "We should exclude commit 100 when returning incremental pull with start commit time as 100"); } + @Test + public void testIncrementalEmptyPartitions() throws IOException { + // initial commit + File partitionDir = InputFormatTestUtil.prepareTable(basePath, baseFileFormat, 10, "100"); + createCommitFile(basePath, "100", "2016/05/01"); + + // Add the paths + FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); + + InputFormatTestUtil.setupIncremental(jobConf, "000", 1); + + FileStatus[] files = inputFormat.listStatus(jobConf); + assertEquals(10, files.length, + "We should include only 1 commit 100 when returning incremental pull with start commit time as 100"); + ensureFilesInCommit("Pulling 1 commits from 000, should get us the 10 files from 100 commit", files, "100", 10); + + // Add new commit only to a new partition + partitionDir = InputFormatTestUtil.prepareTable(basePath, baseFileFormat, 10, "200"); + createCommitFile(basePath, "200", "2017/05/01"); + + InputFormatTestUtil.setupIncremental(jobConf, "100", 1); + files = inputFormat.listStatus(jobConf); + + assertEquals(0, files.length, + "We should exclude commit 200 when returning incremental pull with start commit time as 100 as filePaths does not include new partition"); + } + private void createCommitFile(java.nio.file.Path basePath, String commitNumber, String partitionPath) throws IOException { List writeStats = HoodieTestUtils.generateFakeHoodieWriteStat(1); @@ -355,7 +381,7 @@ public void testGetIncrementalTableNames() throws IOException { String incrementalMode1 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, expectedincrTables[0]); conf.set(incrementalMode1, HoodieHiveUtils.INCREMENTAL_SCAN_MODE); String incrementalMode2 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, expectedincrTables[1]); - conf.set(incrementalMode2,HoodieHiveUtils.INCREMENTAL_SCAN_MODE); + conf.set(incrementalMode2, HoodieHiveUtils.INCREMENTAL_SCAN_MODE); String incrementalMode3 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, "db3.model_trips"); conf.set(incrementalMode3, HoodieHiveUtils.INCREMENTAL_SCAN_MODE.toLowerCase()); String defaultmode = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, "db3.first_trips"); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index fe0be469d28fe..b10f38763d6fa 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -216,7 +216,7 @@ public static HoodieLogFormat.Writer writeRollback(File partitionDir, FileSystem String newCommit, String rolledBackInstant, int logVersion) throws InterruptedException, IOException { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())).withFileId(fileId) - .overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1") + .overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withRolloverLogWriteToken("1-0-1") .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); // generate metadata Map header = new HashMap<>(); @@ -225,7 +225,7 @@ public static HoodieLogFormat.Writer writeRollback(File partitionDir, FileSystem header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); // if update belongs to an existing log file - writer = writer.appendBlock(new HoodieCommandBlock(header)); + writer.appendBlock(new HoodieCommandBlock(header)); return writer; } @@ -235,7 +235,7 @@ public static HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, throws InterruptedException, IOException { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).withLogVersion(logVersion) - .withLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build(); + .withRolloverLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build(); List records = new ArrayList<>(); for (int i = offset; i < offset + numberOfRecords; i++) { records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0")); @@ -245,7 +245,7 @@ public static HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit); header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString()); HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header); - writer = writer.appendBlock(dataBlock); + writer.appendBlock(dataBlock); return writer; } @@ -264,7 +264,7 @@ public static HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionD header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal())); HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header); - writer = writer.appendBlock(rollbackBlock); + writer.appendBlock(rollbackBlock); return writer; } diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index a6cdd08478e1a..ff64ed1242fee 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -142,7 +142,9 @@ Start the Hudi Docker demo: docker/setup_demo.sh ``` -NOTE: We need to make a couple of environment changes for Hive 2.x support. This will be fixed once Hudi moves to Spark 3.x +NOTE: We need to make a couple of environment changes for Hive 2.x support. This will be fixed once Hudi moves to Spark 3.x. +Execute below if you are using Hudi query node in your dag. If not, below section is not required. +Also, for longer running tests, go to next section. ``` docker exec -it adhoc-2 bash @@ -214,7 +216,7 @@ spark-submit \ --conf spark.sql.catalogImplementation=hive \ --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ /opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ ---source-ordering-field timestamp \ +--source-ordering-field test_suite_source_ordering_field \ --use-deltastreamer \ --target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ --input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ @@ -253,7 +255,7 @@ spark-submit \ --conf spark.sql.catalogImplementation=hive \ --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ /opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ ---source-ordering-field timestamp \ +--source-ordering-field test_suite_source_ordering_field \ --use-deltastreamer \ --target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ --input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ @@ -267,3 +269,182 @@ spark-submit \ --table-type MERGE_ON_READ \ --compact-scheduling-minshare 1 ``` + +For long running test suite, validation has to be done differently. Idea is to run same dag in a repeated manner. +Hence "ValidateDatasetNode" is introduced which will read entire input data and compare it with hudi contents both via +spark datasource and hive table via spark sql engine. + +If you have "ValidateDatasetNode" in your dag, do not replace hive jars as instructed above. Spark sql engine does not +go well w/ hive2* jars. So, after running docker setup, just copy test.properties and your dag of interest and you are +good to go ahead. + +For repeated runs, two additional configs need to be set. "dag_rounds" and "dag_intermittent_delay_mins". +This means that your dag will be repeated for N times w/ a delay of Y mins between each round. + +Also, ValidateDatasetNode can be configured in two ways. Either with "delete_input_data: true" set or not set. +When "delete_input_data" is set for ValidateDatasetNode, once validation is complete, entire input data will be deleted. +So, suggestion is to use this ValidateDatasetNode as the last node in the dag with "delete_input_data". +Example dag: +``` + Insert + Upsert + ValidateDatasetNode with delete_input_data = true +``` + +If above dag is run with "dag_rounds" = 10 and "dag_intermittent_delay_mins" = 10, then this dag will run for 10 times +with 10 mins delay between every run. At the end of every run, records written as part of this round will be validated. +At the end of each validation, all contents of input are deleted. +For eg: incase of above dag, +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate with delete_input_data = true + Validates contents from batch1 and batch2 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch1 and batch2. +Round2: + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate with delete_input_data = true + Validates contents from batch3 and batch4 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch3 and batch4. +Round3: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate with delete_input_data = true + Validates contents from batch5 and batch6 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch5 and batch6. +. +. +``` +If you wish to do a cumulative validation, do not set delete_input_data in ValidateDatasetNode. But remember that this +may not scale beyond certain point since input data as well as hudi content's keeps occupying the disk and grows for +every cycle. + +Lets see an example where you don't set "delete_input_data" as part of Validation. +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate: validates contents from batch1 and batch2 are in hudi and ensures Row equality +Round2: + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate: validates contents from batch1 to batch4 are in hudi and ensures Row equality +Round3: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate: validates contents from batch1 and batch6 are in hudi and ensures Row equality +. +. +``` + +You could also have validations in the middle of your dag and not set the "delete_input_data". But set it only in the +last node in the dag. +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate: validates contents from batch1 and batch2 are in hudi and ensures Row equality + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate with delete_input_data = true + Validates contents from batch1 to batch4 are in hudi and ensures Row equality + since "delete_input_data" is set to true, this node deletes contents from batch1 and batch4. +Round2: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate: validates contents from batch5 and batch6 are in hudi and ensures Row equality + insert => inputPath/batch7 + upsert -> inputPath/batch8 + Validate: validates contents from batch5 to batch8 are in hudi and ensures Row equality + since "delete_input_data" is set to true, this node deletes contents from batch5 to batch8. +Round3: + insert => inputPath/batch9 + upsert -> inputPath/batch10 + Validate: validates contents from batch9 and batch10 are in hudi and ensures Row equality + insert => inputPath/batch11 + upsert -> inputPath/batch12 + Validate with delete_input_data = true + Validates contents from batch9 to batch12 are in hudi and ensures Row equality + Set "delete_input_data" to true. so this node deletes contents from batch9 to batch12. +. +. +``` +Above dag was just an example for illustration purposes. But you can make it complex as per your needs. +``` + Insert + Upsert + Delete + Validate w/o deleting + Insert + Rollback + Validate w/o deleting + Upsert + Validate w/ deletion +``` +With this dag, you can set the two additional configs "dag_rounds" and "dag_intermittent_delay_mins" and have a long +running test suite. + +``` +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + Insert + Upsert + Delete + Validate w/o deleting + Insert + Rollback + Validate w/o deleting + Upsert + Validate w/ deletion + +``` + +Sample COW command with repeated runs. +``` +spark-submit \ +--packages org.apache.spark:spark-avro_2.11:2.4.0 \ +--conf spark.task.cpus=1 \ +--conf spark.executor.cores=1 \ +--conf spark.task.maxFailures=100 \ +--conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true \ +--conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 \ +--conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false \ +--conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s \ +--conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 \ +--conf spark.sql.catalogImplementation=hive \ +--conf spark.driver.extraClassPath=/var/demo/jars/* \ +--conf spark.executor.extraClassPath=/var/demo/jars/* \ +--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ +/opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ +--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ +--target-table table1 \ +--props test.properties \ +--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-path file:/var/hoodie/ws/docker/demo/config/test-suite/complex-dag-cow.yaml \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 +``` + +A ready to use dag is available under docker/demo/config/test-suite/ that could give you an idea for long running +dags. +cow-per-round-mixed-validate.yaml + +As of now, "ValidateDatasetNode" uses spark data source and hive tables for comparison. Hence COW and real time view in +MOR can be tested. + + \ No newline at end of file diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index e9fcc61e013b5..b48dd813584c8 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml hudi-integ-test @@ -206,12 +206,11 @@ com.fasterxml.jackson.dataformat jackson-dataformat-yaml - 2.7.4 + ${fasterxml.jackson.dataformat.yaml.version} com.fasterxml.jackson.core jackson-databind - 2.6.7.3 @@ -220,11 +219,6 @@ jackson-annotations test - - com.fasterxml.jackson.core - jackson-databind - test - com.fasterxml.jackson.datatype jackson-datatype-guava diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java index 0387731d6b469..8d2f79defa8f2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java @@ -25,13 +25,13 @@ import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.SchemaProvider; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; /** - * Extends the {@link HoodieDeltaStreamer} to expose certain operations helpful in running the Test Suite. - * This is done to achieve 2 things 1) Leverage some components of {@link HoodieDeltaStreamer} 2) - * Piggyback on the suite to test {@link HoodieDeltaStreamer} + * Extends the {@link HoodieDeltaStreamer} to expose certain operations helpful in running the Test Suite. This is done to achieve 2 things 1) Leverage some components of {@link HoodieDeltaStreamer} + * 2) Piggyback on the suite to test {@link HoodieDeltaStreamer} */ public class HoodieDeltaStreamerWrapper extends HoodieDeltaStreamer { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index 7b3324e4b569e..b5037e9959cb3 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -18,13 +18,6 @@ package org.apache.hudi.integ.testsuite; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import java.io.IOException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hudi.DataSourceUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; @@ -35,23 +28,30 @@ import org.apache.hudi.integ.testsuite.dag.DagUtils; import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator; +import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; -import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.keygen.BuiltinKeyGenerator; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; -import org.apache.hudi.utilities.schema.SchemaProvider; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; + /** - * This is the entry point for running a Hudi Test Suite. Although this class has similarities with - * {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency on the changes in - * DeltaStreamer. + * This is the entry point for running a Hudi Test Suite. Although this class has similarities with {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency + * on the changes in DeltaStreamer. */ public class HoodieTestSuiteJob { @@ -133,10 +133,10 @@ public static void main(String[] args) throws Exception { public WorkflowDag createWorkflowDag() throws IOException { WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils - .loadClass((this.cfg).workloadDagGenerator)).build() - : DagUtils.convertYamlPathToDag( - FSUtils.getFs(this.cfg.workloadYamlPath, jsc.hadoopConfiguration(), true), - this.cfg.workloadYamlPath); + .loadClass((this.cfg).workloadDagGenerator)).build() + : DagUtils.convertYamlPathToDag( + FSUtils.getFs(this.cfg.workloadYamlPath, jsc.hadoopConfiguration(), true), + this.cfg.workloadYamlPath); return workflowDag; } @@ -147,7 +147,7 @@ public void runTestSuite() { long startTime = System.currentTimeMillis(); WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); - DagScheduler dagScheduler = new DagScheduler(workflowDag, writerContext); + DagScheduler dagScheduler = new DagScheduler(workflowDag, writerContext, jsc); dagScheduler.schedule(); log.info("Finished scheduling all tasks, Time taken {}", System.currentTimeMillis() - startTime); } catch (Exception e) { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java index bf6fca70ecea4..a06c281ef719e 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java @@ -19,6 +19,7 @@ package org.apache.hudi.integ.testsuite; import org.apache.hadoop.conf.Configuration; + import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; @@ -31,6 +32,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; @@ -98,6 +100,8 @@ private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteConfig cfg, Prope HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath) .withAutoCommit(false) .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build()) + .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) + .build()) .forTable(cfg.targetTableName) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withProps(props); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java index 81f406be3cc32..329ef16bd50d2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java @@ -87,6 +87,7 @@ public static class Config { private static String HIVE_LOCAL = "hive_local"; private static String REINIT_CONTEXT = "reinitialize_context"; private static String START_PARTITION = "start_partition"; + private static String DELETE_INPUT_DATA = "delete_input_data"; private Map configsMap; @@ -154,6 +155,10 @@ public boolean getReinitContext() { return Boolean.valueOf(configsMap.getOrDefault(REINIT_CONTEXT, false).toString()); } + public boolean isDeleteInputData() { + return Boolean.valueOf(configsMap.getOrDefault(DELETE_INPUT_DATA, false).toString()); + } + public Map getOtherConfigs() { if (configsMap == null) { return new HashMap<>(); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java index 24520a3626cf3..1e8acf580d962 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java @@ -18,12 +18,14 @@ package org.apache.hudi.integ.testsuite.converter; -import java.util.List; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.integ.testsuite.generator.LazyRecordGeneratorIterator; import org.apache.hudi.integ.testsuite.generator.UpdateGeneratorIterator; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; +import java.util.List; + /** * This converter creates an update {@link GenericRecord} from an existing {@link GenericRecord}. */ @@ -36,7 +38,7 @@ public class UpdateConverter implements Converter private final List recordKeyFields; private final int minPayloadSize; - public UpdateConverter(String schemaStr, int minPayloadSize, List partitionPathFields, + public UpdateConverter(String schemaStr, int minPayloadSize, List partitionPathFields, List recordKeyFields) { this.schemaStr = schemaStr; this.partitionPathFields = partitionPathFields; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java index d5358238d51a1..1211c0098d23a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java @@ -48,6 +48,15 @@ */ public class DagUtils { + public static final String DAG_NAME = "dag_name"; + public static final String DAG_ROUNDS = "dag_rounds"; + public static final String DAG_INTERMITTENT_DELAY_MINS = "dag_intermittent_delay_mins"; + public static final String DAG_CONTENT = "dag_content"; + + public static int DEFAULT_DAG_ROUNDS = 1; + public static int DEFAULT_INTERMITTENT_DELAY_MINS = 10; + public static String DEFAULT_DAG_NAME = "TestDagName"; + static final ObjectMapper MAPPER = new ObjectMapper(); /** @@ -62,15 +71,38 @@ public static WorkflowDag convertYamlPathToDag(FileSystem fs, String path) throw * Converts a YAML representation to {@link WorkflowDag}. */ public static WorkflowDag convertYamlToDag(String yaml) throws IOException { + int dagRounds = DEFAULT_DAG_ROUNDS; + int intermittentDelayMins = DEFAULT_INTERMITTENT_DELAY_MINS; + String dagName = DEFAULT_DAG_NAME; Map allNodes = new HashMap<>(); final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); final JsonNode jsonNode = yamlReader.readTree(yaml); Iterator> itr = jsonNode.fields(); while (itr.hasNext()) { Entry dagNode = itr.next(); - allNodes.put(dagNode.getKey(), convertJsonToDagNode(allNodes, dagNode.getKey(), dagNode.getValue())); + String key = dagNode.getKey(); + switch (key) { + case DAG_NAME: + dagName = dagNode.getValue().asText(); + break; + case DAG_ROUNDS: + dagRounds = dagNode.getValue().asInt(); + break; + case DAG_INTERMITTENT_DELAY_MINS: + intermittentDelayMins = dagNode.getValue().asInt(); + break; + case DAG_CONTENT: + JsonNode dagContent = dagNode.getValue(); + Iterator> contentItr = dagContent.fields(); + while(contentItr.hasNext()) { + Entry dagContentNode = contentItr.next(); + allNodes.put(dagContentNode.getKey(), convertJsonToDagNode(allNodes, dagContentNode.getKey(), dagContentNode.getValue())); + } + default: + break; + } } - return new WorkflowDag(findRootNodes(allNodes)); + return new WorkflowDag(dagName, dagRounds, intermittentDelayMins, findRootNodes(allNodes)); } /** diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java index ad6e9cb0cb1b9..1fe2294423121 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java @@ -18,8 +18,6 @@ package org.apache.hudi.integ.testsuite.dag; -import java.util.ArrayList; -import java.util.List; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; @@ -27,9 +25,11 @@ import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode; import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode; +import java.util.ArrayList; +import java.util.List; + /** - * An example of how to generate a workflow dag programmatically. This is also used as the default workflow dag if - * none is provided. + * An example of how to generate a workflow dag programmatically. This is also used as the default workflow dag if none is provided. */ public class SimpleWorkflowDagGenerator implements WorkflowDagGenerator { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java index e9171fc4774d0..f622bb7a7e448 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java @@ -18,20 +18,47 @@ package org.apache.hudi.integ.testsuite.dag; -import java.util.List; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; +import java.util.List; + +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_DAG_NAME; +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_DAG_ROUNDS; +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_INTERMITTENT_DELAY_MINS; + /** * Workflow dag that encapsulates all execute nodes. */ public class WorkflowDag { + private String dagName; + private int rounds; + private int intermittentDelayMins; private List> nodeList; public WorkflowDag(List> nodeList) { + this(DEFAULT_DAG_NAME, DEFAULT_DAG_ROUNDS, DEFAULT_INTERMITTENT_DELAY_MINS, nodeList); + } + + public WorkflowDag(String dagName, int rounds, int intermittentDelayMins, List> nodeList) { + this.dagName = dagName; + this.rounds = rounds; + this.intermittentDelayMins = intermittentDelayMins; this.nodeList = nodeList; } + public String getDagName() { + return dagName; + } + + public int getRounds() { + return rounds; + } + + public int getIntermittentDelayMins() { + return intermittentDelayMins; + } + public List> getNodeList() { return nodeList; } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index e457f0a8daca7..650ab1eeaf4c2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -21,15 +21,16 @@ import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; +import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; import org.apache.hudi.keygen.BuiltinKeyGenerator; -import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; -import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -38,8 +39,7 @@ import java.util.Map; /** - * WriterContext wraps the delta writer/data generator related configuration needed - * to init/reinit. + * WriterContext wraps the delta writer/data generator related configuration needed to init/reinit. */ public class WriterContext { @@ -53,8 +53,9 @@ public class WriterContext { private BuiltinKeyGenerator keyGenerator; private transient SparkSession sparkSession; private transient JavaSparkContext jsc; + public WriterContext(JavaSparkContext jsc, TypedProperties props, HoodieTestSuiteConfig cfg, - BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { + BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { this.cfg = cfg; this.props = props; this.keyGenerator = keyGenerator; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java index df54b4c811989..05ac242a5cefc 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java @@ -41,6 +41,17 @@ public abstract class DagNode implements Comparable> { protected Config config; private boolean isCompleted; + public DagNode clone() { + List> tempChildNodes = new ArrayList<>(); + for(DagNode dagNode: childNodes) { + tempChildNodes.add(dagNode.clone()); + } + this.childNodes = tempChildNodes; + this.result = null; + this.isCompleted = false; + return this; + } + public DagNode addChildNode(DagNode childNode) { childNode.getParentNodes().add(this); getChildNodes().add(childNode); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java new file mode 100644 index 0000000000000..c0671e8abb1ef --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes; + +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Delay Node to add delays between each group of test runs. + */ +public class DelayNode extends DagNode { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + private int delayMins; + + public DelayNode(int delayMins) { + this.delayMins = delayMins; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + log.warn("Waiting for "+ delayMins+" mins before going for next test run"); + Thread.sleep(delayMins * 60 * 1000); + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java new file mode 100644 index 0000000000000..12fc52529a0ab --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java @@ -0,0 +1,148 @@ +/* + * 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.integ.testsuite.dag.nodes; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; + +/** + * This nodes validates contents from input path are in tact with Hudi. This nodes uses spark datasource for comparison purposes. By default no configs are required for this node. But there is an + * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. + * README has more details under docker set up for usages of this node. + */ +public class ValidateDatasetNode extends DagNode { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + + public ValidateDatasetNode(Config config) { + this.config = config; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + + SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); + + // todo: Fix partitioning schemes. For now, assumes data based partitioning. + String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; + String hudiPath = context.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/*/*/*"; + log.warn("ValidateDataset Node: Input path " + inputPath + ", hudi path " + hudiPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } + + String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); + String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()); + // todo: fix hard coded fields from configs. + // read input and resolve insert, updates, etc. + Dataset inputDf = session.read().format("avro").load(inputPath); + ExpressionEncoder encoder = getEncoder(inputDf.schema()); + Dataset inputSnapshotDf = inputDf.groupByKey( + (MapFunction) value -> partitionPathField + "+" + recordKeyField, Encoders.STRING()) + .reduceGroups((ReduceFunction) (v1, v2) -> { + int ts1 = v1.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + int ts2 = v2.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + if (ts1 > ts2) { + return v1; + } else { + return v2; + } + }) + .map((MapFunction, Row>) value -> value._2, encoder) + .filter("_hoodie_is_deleted is NULL"); + + // read from hudi and remove meta columns. + Dataset hudiDf = session.read().format("hudi").load(hudiPath); + Dataset trimmedDf = hudiDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD); + + Dataset intersectionDf = inputSnapshotDf.intersect(trimmedDf); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed. Total count in hudi " + trimmedDf.count() + ", input df count " + inputSnapshotDf.count()); + throw new AssertionError("Hudi contents does not match contents input data. "); + } + + String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY()); + String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()); + log.warn("Validating hive table with db : " + database + " and table : " + tableName); + Dataset cowDf = session.sql("SELECT * FROM " + database + "." + tableName); + Dataset trimmedCowDf = cowDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD); + intersectionDf = inputSnapshotDf.intersect(trimmedDf); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed for COW hive table. Total count in hudi " + trimmedCowDf.count() + ", input df count " + inputSnapshotDf.count()); + throw new AssertionError("Hudi hive table contents does not match contents input data. "); + } + + // if delete input data is enabled, erase input data. + if (config.isDeleteInputData()) { + // clean up input data for current group of writes. + inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Micro batch to be deleted " + fileStatus.getPath().toString()); + fs.delete(fileStatus.getPath(), true); + } + } + } + + private ExpressionEncoder getEncoder(StructType schema) { + List attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream() + .map(Attribute::toAttribute).collect(Collectors.toList()); + return RowEncoder.apply(schema) + .resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(), + SimpleAnalyzer$.MODULE$); + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java index 5c70ea164e873..d4074bcccdae0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java @@ -23,8 +23,10 @@ import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; +import org.apache.hudi.integ.testsuite.dag.nodes.DelayNode; import org.apache.hudi.metrics.Metrics; +import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,9 +52,9 @@ public class DagScheduler { private WorkflowDag workflowDag; private ExecutionContext executionContext; - public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext) { + public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext, JavaSparkContext jsc) { this.workflowDag = workflowDag; - this.executionContext = new ExecutionContext(null, writerContext); + this.executionContext = new ExecutionContext(jsc, writerContext); } /** @@ -63,7 +65,7 @@ public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext) { public void schedule() throws Exception { ExecutorService service = Executors.newFixedThreadPool(2); try { - execute(service, workflowDag.getNodeList()); + execute(service, workflowDag); service.shutdown(); } finally { if (!service.isShutdown()) { @@ -77,33 +79,47 @@ public void schedule() throws Exception { * Method to start executing the nodes in workflow DAGs. * * @param service ExecutorService - * @param nodes Nodes to be executed + * @param workflowDag instance of workflow dag that needs to be executed * @throws Exception will be thrown if ant error occurred */ - private void execute(ExecutorService service, List nodes) throws Exception { + private void execute(ExecutorService service, WorkflowDag workflowDag) throws Exception { // Nodes at the same level are executed in parallel - Queue queue = new PriorityQueue<>(nodes); log.info("Running workloads"); + List nodes = workflowDag.getNodeList(); + int curRound = 1; do { - List futures = new ArrayList<>(); - Set childNodes = new HashSet<>(); - while (queue.size() > 0) { - DagNode nodeToExecute = queue.poll(); - log.info("Node to execute in dag scheduler " + nodeToExecute.getConfig().toString()); - futures.add(service.submit(() -> executeNode(nodeToExecute))); - if (nodeToExecute.getChildNodes().size() > 0) { - childNodes.addAll(nodeToExecute.getChildNodes()); - } + log.warn("==================================================================="); + log.warn("Running workloads for round num " + curRound); + log.warn("==================================================================="); + Queue queue = new PriorityQueue<>(); + for (DagNode dagNode : nodes) { + queue.add(dagNode.clone()); } - queue.addAll(childNodes); - childNodes.clear(); - for (Future future : futures) { - future.get(1, TimeUnit.HOURS); + do { + List futures = new ArrayList<>(); + Set childNodes = new HashSet<>(); + while (queue.size() > 0) { + DagNode nodeToExecute = queue.poll(); + log.warn("Executing node \"" + nodeToExecute.getConfig().getOtherConfigs().get(CONFIG_NAME) + "\" :: " + nodeToExecute.getConfig()); + futures.add(service.submit(() -> executeNode(nodeToExecute))); + if (nodeToExecute.getChildNodes().size() > 0) { + childNodes.addAll(nodeToExecute.getChildNodes()); + } + } + queue.addAll(childNodes); + childNodes.clear(); + for (Future future : futures) { + future.get(1, TimeUnit.HOURS); + } + } while (queue.size() > 0); + log.info("Finished workloads for round num " + curRound); + if (curRound < workflowDag.getRounds()) { + new DelayNode(workflowDag.getIntermittentDelayMins()).execute(executionContext); } // After each level, report and flush the metrics Metrics.flush(); - } while (queue.size() > 0); + } while (curRound++ < workflowDag.getRounds()); log.info("Finished workloads"); } @@ -119,7 +135,6 @@ private void executeNode(DagNode node) { try { int repeatCount = node.getConfig().getRepeatCount(); while (repeatCount > 0) { - log.warn("executing node: \"" + node.getConfig().getOtherConfigs().get(CONFIG_NAME) + "\" of type: " + node.getClass() + " :: " + node.getConfig().toString()); node.execute(executionContext); log.info("Finished executing {}", node.getName()); repeatCount--; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index 53af8eb74068e..30b2d6ce0dfa0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -41,6 +41,10 @@ import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.converter.Converter; import org.apache.hudi.integ.testsuite.converter.DeleteConverter; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.converter.UpdateConverter; import org.apache.hudi.integ.testsuite.reader.DFSAvroDeltaInputReader; import org.apache.hudi.integ.testsuite.reader.DFSHoodieDatasetInputReader; @@ -51,6 +55,7 @@ import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory; import org.apache.hudi.keygen.BuiltinKeyGenerator; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; @@ -58,6 +63,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.StreamSupport; import scala.Tuple2; @@ -77,7 +93,7 @@ public class DeltaGenerator implements Serializable { private int batchId; public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession, - String schemaStr, BuiltinKeyGenerator keyGenerator) { + String schemaStr, BuiltinKeyGenerator keyGenerator) { this.deltaOutputConfig = deltaOutputConfig; this.jsc = jsc; this.sparkSession = sparkSession; @@ -113,7 +129,7 @@ public JavaRDD writeRecords(JavaRDD records) { public JavaRDD generateInserts(Config operation) { int numPartitions = operation.getNumInsertPartitions(); - long recordsPerPartition = operation.getNumRecordsInsert() / numPartitions; + long recordsPerPartition = operation.getNumRecordsInsert(); int minPayloadSize = operation.getRecordSize(); int startPartition = operation.getStartPartition(); @@ -124,7 +140,7 @@ public JavaRDD generateInserts(Config operation) { JavaRDD inputBatch = jsc.parallelize(partitionIndexes, numPartitions) .mapPartitionsWithIndex((index, p) -> { return new LazyRecordGeneratorIterator(new FlexibleSchemaRecordGenerationIterator(recordsPerPartition, - minPayloadSize, schemaStr, partitionPathFieldNames, (Integer)index)); + minPayloadSize, schemaStr, partitionPathFieldNames, numPartitions)); }, true); if (deltaOutputConfig.getInputParallelism() < numPartitions) { @@ -167,7 +183,6 @@ public JavaRDD generateUpdates(Config config) throws IOException log.info("Repartitioning records into " + numPartition + " partitions for updates"); adjustedRDD = adjustedRDD.repartition(numPartition); log.info("Repartitioning records done for updates"); - UpdateConverter converter = new UpdateConverter(schemaStr, config.getRecordSize(), partitionPathFieldNames, recordRowKeyFieldNames); JavaRDD updates = converter.convert(adjustedRDD); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java index 256dfa49ebf6c..787ec844edbc7 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java @@ -41,17 +41,21 @@ public class FlexibleSchemaRecordGenerationIterator implements Iterator partitionPathFieldNames; + private String firstPartitionPathField; public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, String schema) { this(maxEntriesToProduce, GenericRecordFullPayloadGenerator.DEFAULT_PAYLOAD_SIZE, schema, null, 0); } public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, int minPayloadSize, String schemaStr, - List partitionPathFieldNames, int partitionIndex) { + List partitionPathFieldNames, int numPartitions) { this.counter = maxEntriesToProduce; this.partitionPathFieldNames = new HashSet<>(partitionPathFieldNames); + if(partitionPathFieldNames != null && partitionPathFieldNames.size() > 0) { + this.firstPartitionPathField = partitionPathFieldNames.get(0); + } Schema schema = new Schema.Parser().parse(schemaStr); - this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize, partitionIndex); + this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize, numPartitions); } @Override @@ -62,12 +66,18 @@ public boolean hasNext() { @Override public GenericRecord next() { this.counter--; + boolean partitionPathsNonEmpty = partitionPathFieldNames != null && partitionPathFieldNames.size() > 0; if (lastRecord == null) { - GenericRecord record = this.generator.getNewPayload(partitionPathFieldNames); + GenericRecord record = partitionPathsNonEmpty + ? this.generator.getNewPayloadWithTimestamp(this.firstPartitionPathField) + : this.generator.getNewPayload(); lastRecord = record; return record; } else { - return this.generator.randomize(lastRecord, partitionPathFieldNames); + return partitionPathsNonEmpty + ? this.generator.getUpdatePayloadWithTimestamp(lastRecord, + this.partitionPathFieldNames, firstPartitionPathField) + : this.generator.getUpdatePayload(lastRecord, this.partitionPathFieldNames); } } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java index 7d5ca081444f0..510fc499b9a2c 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java @@ -46,9 +46,9 @@ */ public class GenericRecordFullPayloadGenerator implements Serializable { - private static Logger LOG = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class); - + private static final Logger LOG = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class); public static final int DEFAULT_PAYLOAD_SIZE = 1024 * 10; // 10 KB + public static final int DEFAULT_NUM_DATE_PARTITIONS = 50; public static final String DEFAULT_HOODIE_IS_DELETED_COL = "_hoodie_is_deleted"; protected final Random random = new Random(); // The source schema used to generate a payload @@ -58,10 +58,12 @@ public class GenericRecordFullPayloadGenerator implements Serializable { // The index of partition for which records are being generated private int partitionIndex = 0; // The size of a full record where every field of a generic record created contains 1 random value - private final int estimatedFullPayloadSize; + private int estimatedFullPayloadSize; // Number of extra entries to add in a complex/collection field to achieve the desired record size Map extraEntriesMap = new HashMap<>(); + // The number of unique dates to create + private int numDatePartitions = DEFAULT_NUM_DATE_PARTITIONS; // LogicalTypes in Avro 1.8.2 private static final String DECIMAL = "decimal"; private static final String UUID_NAME = "uuid"; @@ -75,6 +77,11 @@ public GenericRecordFullPayloadGenerator(Schema schema) { this(schema, DEFAULT_PAYLOAD_SIZE); } + public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize, int numDatePartitions) { + this(schema, minPayloadSize); + this.numDatePartitions = numDatePartitions; + } + public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize) { Pair sizeInfo = new GenericRecordFullPayloadSizeEstimator(schema) .typeEstimateAndNumComplexFields(); @@ -83,19 +90,13 @@ public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize) { if (estimatedFullPayloadSize < minPayloadSize) { int numberOfComplexFields = sizeInfo.getRight(); if (numberOfComplexFields < 1) { - LOG.warn("The schema does not have any collections/complex fields. " - + "Cannot achieve minPayloadSize => " + minPayloadSize); + LOG.warn("The schema does not have any collections/complex fields. Cannot achieve minPayloadSize : {}", + minPayloadSize); } - determineExtraEntriesRequired(numberOfComplexFields, minPayloadSize - estimatedFullPayloadSize); } } - public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize, int partitionIndex) { - this(schema, minPayloadSize); - this.partitionIndex = partitionIndex; - } - protected static boolean isPrimitive(Schema localSchema) { if (localSchema.getType() != Type.ARRAY && localSchema.getType() != Type.MAP @@ -131,15 +132,28 @@ public GenericRecord getNewPayload(Set partitionPathFieldNames) { return create(baseSchema, partitionPathFieldNames); } + public GenericRecord getNewPayloadWithTimestamp(String tsFieldName) { + return updateTimestamp(create(baseSchema, null), tsFieldName); + } + + public GenericRecord getUpdatePayloadWithTimestamp(GenericRecord record, Set blacklistFields, + String tsFieldName) { + return updateTimestamp(randomize(record, blacklistFields), tsFieldName); + } + protected GenericRecord create(Schema schema, Set partitionPathFieldNames) { GenericRecord result = new GenericData.Record(schema); for (Schema.Field f : schema.getFields()) { - if (isPartialLongField(f, partitionPathFieldNames)) { - // This is a long field used as partition field. Set it to seconds since epoch. - long value = TimeUnit.SECONDS.convert(partitionIndex, TimeUnit.DAYS); - result.put(f.name(), (long) value); + if (f.name().equals(DEFAULT_HOODIE_IS_DELETED_COL)) { + result.put(f.name(), false); } else { - result.put(f.name(), typeConvert(f)); + if (isPartialLongField(f, partitionPathFieldNames)) { + // This is a long field used as partition field. Set it to seconds since epoch. + long value = TimeUnit.SECONDS.convert(partitionIndex, TimeUnit.DAYS); + result.put(f.name(), (long) value); + } else { + result.put(f.name(), typeConvert(f)); + } } } return result; @@ -310,6 +324,17 @@ public boolean validate(GenericRecord record) { return genericData.validate(baseSchema, record); } + /* + * Generates a sequential timestamp (daily increment), and updates the timestamp field of the record. + * Note: When generating records, number of records to be generated must be more than numDatePartitions * parallelism, + * to guarantee that at least numDatePartitions are created. + */ + public GenericRecord updateTimestamp(GenericRecord record, String fieldName) { + long delta = TimeUnit.MILLISECONDS.convert(++partitionIndex % numDatePartitions, TimeUnit.DAYS); + record.put(fieldName, System.currentTimeMillis() - delta); + return record; + } + /** * Check whether a schema is option. return true if it match the follows: 1. Its type is Type.UNION 2. Has two types 3. Has a NULL type. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java index 51b1fd9ed48dc..89cda658e12cc 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java @@ -18,16 +18,16 @@ package org.apache.hudi.integ.testsuite.generator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + /** * A lazy update payload generator to generate {@link GenericRecord}s lazily. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java index b7d71f583777a..94ff3a3ea2327 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java @@ -18,20 +18,6 @@ package org.apache.hudi.integ.testsuite.helpers; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FsStatus; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -40,13 +26,26 @@ import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob; import org.apache.hudi.utilities.sources.helpers.DFSPathSelector; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + /** * A custom dfs path selector used only for the hudi test suite. To be used only if workload is not run inline. */ public class DFSTestSuitePathSelector extends DFSPathSelector { + private static volatile Logger log = LoggerFactory.getLogger(HoodieTestSuiteJob.class); public DFSTestSuitePathSelector(TypedProperties props, Configuration hadoopConf) { @@ -67,6 +66,7 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( lastBatchId = 0; nextBatchId = 1; } + // obtain all eligible files for the batch List eligibleFiles = new ArrayList<>(); FileStatus[] fileStatuses = fs.globStatus( @@ -87,7 +87,8 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( if (!fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream() .anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) { continue; - } else if (fileStatus.getPath().getName().compareTo(lastBatchId.toString()) > 0) { + } else if (Integer.parseInt(fileStatus.getPath().getName()) > lastBatchId && Integer.parseInt(fileStatus.getPath() + .getName()) <= nextBatchId) { RemoteIterator files = fs.listFiles(fileStatus.getPath(), true); while (files.hasNext()) { eligibleFiles.add(files.next()); @@ -95,7 +96,6 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( } } - log.info("Reading " + eligibleFiles.size() + " files. "); // no data to readAvro if (eligibleFiles.size() == 0) { return new ImmutablePair<>(Option.empty(), diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java index da9268161d4d8..ad6ef10463009 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java @@ -80,7 +80,6 @@ protected Pair getFileStatusIndexRange(List fileSt } if (totalSizeOfFilesPresent <= totalSizeToRead) { endOffset++; - continue; } else { return Pair.of(startOffset, endOffset); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index cfe7991f43ee5..136aa27777f66 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -18,27 +18,9 @@ package org.apache.hudi.integ.testsuite.reader; -import static java.util.Map.Entry.comparingByValue; -import static java.util.stream.Collectors.toMap; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; @@ -51,18 +33,41 @@ import org.apache.hudi.common.util.ParquetReaderIterator; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieMemoryConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroReadSupport; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + import scala.Tuple2; +import static java.util.Map.Entry.comparingByValue; +import static java.util.stream.Collectors.toMap; + /** - * This class helps to generate updates from an already existing hoodie dataset. It supports generating updates in - * across partitions, files and records. + * This class helps to generate updates from an already existing hoodie dataset. It supports generating updates in across partitions, files and records. */ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader { @@ -81,8 +86,9 @@ public DFSHoodieDatasetInputReader(JavaSparkContext jsc, String basePath, String protected List getPartitions(Option partitionsLimit) throws IOException { // Using FSUtils.getFS here instead of metaClient.getFS() since we dont want to count these listStatus // calls in metrics as they are not part of normal HUDI operation. - FileSystem fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf()); - List partitionPaths = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath(), false); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + List partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getBasePath(), + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false); // Sort partition so we can pick last N partitions by default Collections.sort(partitionPaths); if (!partitionPaths.isEmpty()) { @@ -147,16 +153,22 @@ private JavaRDD fetchRecordsFromDataset(Option numPartit long recordsInSingleFile = iteratorSize(readParquetOrLogFiles(getSingleSliceFromRDD(partitionToFileSlice))); int numFilesToUpdate; long numRecordsToUpdatePerFile; - if (!numFiles.isPresent() || numFiles.get() == 0) { + if (!numFiles.isPresent() || numFiles.get() <= 0) { // If num files are not passed, find the number of files to update based on total records to update and records // per file - numFilesToUpdate = (int)Math.ceil((double)numRecordsToUpdate.get() / recordsInSingleFile); - // recordsInSingleFile is not average so we still need to account for bias is records distribution - // in the files. Limit to the maximum number of files available. - int totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get(); - numFilesToUpdate = Math.min(numFilesToUpdate, totalExistingFilesCount); - log.info("Files to update {}", numFilesToUpdate); - numRecordsToUpdatePerFile = recordsInSingleFile; + numFilesToUpdate = (int) Math.floor((double) numRecordsToUpdate.get() / recordsInSingleFile); + if (numFilesToUpdate > 0) { + // recordsInSingleFile is not average so we still need to account for bias is records distribution + // in the files. Limit to the maximum number of files available. + int totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get(); + numFilesToUpdate = Math.min(numFilesToUpdate, totalExistingFilesCount); + log.info("Files to update {}, records to update per file {}", numFilesToUpdate, recordsInSingleFile); + numRecordsToUpdatePerFile = recordsInSingleFile; + } else { + numFilesToUpdate = 1; + numRecordsToUpdatePerFile = numRecordsToUpdate.get(); + log.info("Total records passed in < records in single file. Hence setting numFilesToUpdate to 1 and numRecordsToUpdate to {} ", numRecordsToUpdatePerFile); + } } else { // If num files is passed, find the number of records per file based on either percentage or total records to // update and num files passed @@ -170,6 +182,7 @@ private JavaRDD fetchRecordsFromDataset(Option numPartit partitionPaths.size(), numFilesToUpdate, partitionToFileIdCountMap); JavaRDD updates = projectSchema(generateUpdates(adjustedPartitionToFileIdCountMap, partitionToFileSlice, numFilesToUpdate, (int) numRecordsToUpdatePerFile)); + if (numRecordsToUpdate.isPresent() && numFiles.isPresent() && numFiles.get() != 0 && numRecordsToUpdate.get() != numRecordsToUpdatePerFile * numFiles.get()) { long remainingRecordsToAdd = (numRecordsToUpdate.get() - (numRecordsToUpdatePerFile * numFiles.get())); @@ -183,7 +196,7 @@ private JavaRDD fetchRecordsFromDataset(Option numPartit private JavaRDD projectSchema(JavaRDD updates) { // The records read from the hoodie dataset have the hoodie record fields, rewrite the record to eliminate them return updates - .map(r -> HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(r, new Schema.Parser().parse(schemaStr))); + .map(r -> HoodieAvroUtils.rewriteRecord(r, new Schema.Parser().parse(schemaStr))); } private JavaRDD generateUpdates(Map adjustedPartitionToFileIdCountMap, @@ -214,7 +227,7 @@ private Map getFilesToReadPerPartition(JavaPairRDD adjustedPartitionToFileIdCountMap = new HashMap<>(); partitionToFileIdCountSortedMap.entrySet().stream().forEach(e -> { if (e.getValue() <= numFilesPerPartition) { @@ -243,20 +256,30 @@ private FileSlice getSingleSliceFromRDD(JavaPairRDD> private Iterator readParquetOrLogFiles(FileSlice fileSlice) throws IOException { if (fileSlice.getBaseFile().isPresent()) { + // Read the parquet files using the latest writer schema. + Schema schema = new Schema.Parser().parse(schemaStr); + AvroReadSupport.setAvroReadSchema(metaClient.getHadoopConf(), HoodieAvroUtils.addMetadataFields(schema)); Iterator itr = new ParquetReaderIterator(AvroParquetReader.builder(new Path(fileSlice.getBaseFile().get().getPath())).withConf(metaClient.getHadoopConf()).build()); return itr; } else { // If there is no data file, fall back to reading log files - HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(metaClient.getFs(), - metaClient.getBasePath(), - fileSlice.getLogFiles().map(l -> l.getPath().getName()).collect(Collectors.toList()), - new Schema.Parser().parse(schemaStr), metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().lastInstant().get().getTimestamp(), - HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, true, false, - HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE, - HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH); + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(metaClient.getFs()) + .withBasePath(metaClient.getBasePath()) + .withLogFilePaths( + fileSlice.getLogFiles().map(l -> l.getPath().getName()).collect(Collectors.toList())) + .withReaderSchema(new Schema.Parser().parse(schemaStr)) + .withLatestInstantTime(metaClient.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().lastInstant().get().getTimestamp()) + .withMaxMemorySizeInBytes( + HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES) + .withReadBlocksLazily(true) + .withReverseReader(false) + .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE) + .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH) + .build(); // readAvro log files Iterable> iterable = () -> scanner.iterator(); Schema schema = new Schema.Parser().parse(schemaStr); @@ -272,9 +295,7 @@ private Iterator readParquetOrLogFiles(FileSlice fileSlice) throw } /** - * Returns the number of elements remaining in {@code iterator}. The iterator - * will be left exhausted: its {@code hasNext()} method will return - * {@code false}. + * Returns the number of elements remaining in {@code iterator}. The iterator will be left exhausted: its {@code hasNext()} method will return {@code false}. */ private static int iteratorSize(Iterator iterator) { int count = 0; @@ -286,11 +307,8 @@ private static int iteratorSize(Iterator iterator) { } /** - * Creates an iterator returning the first {@code limitSize} elements of the - * given iterator. If the original iterator does not contain that many - * elements, the returned iterator will have the same behavior as the original - * iterator. The returned iterator supports {@code remove()} if the original - * iterator does. + * Creates an iterator returning the first {@code limitSize} elements of the given iterator. If the original iterator does not contain that many elements, the returned iterator will have the same + * behavior as the original iterator. The returned iterator supports {@code remove()} if the original iterator does. * * @param iterator the iterator to limit * @param limitSize the maximum number of elements in the returned iterator diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java index 16a5259ee94ba..fc23a47b35331 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java @@ -20,7 +20,7 @@ import java.util.List; import org.apache.avro.generic.GenericRecord; -import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.utilities.schema.RowBasedSchemaProvider; import org.apache.spark.api.java.JavaRDD; @@ -49,7 +49,7 @@ public static JavaRDD readAvro(SparkSession sparkSession, String .option(AVRO_SCHEMA_OPTION_KEY, schemaStr) .load(JavaConverters.asScalaIteratorConverter(listOfPaths.iterator()).asScala().toSeq()); - return AvroConversionUtils + return HoodieSparkUtils .createRdd(dataSet.toDF(), structName.orElse(RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME), nameSpace.orElse(RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE)) .toJavaRDD(); @@ -61,7 +61,7 @@ public static JavaRDD readParquet(SparkSession sparkSession, List Dataset dataSet = sparkSession.read() .parquet((JavaConverters.asScalaIteratorConverter(listOfPaths.iterator()).asScala().toSeq())); - return AvroConversionUtils + return HoodieSparkUtils .createRdd(dataSet.toDF(), structName.orElse(RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME), RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE) .toJavaRDD(); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java new file mode 100644 index 0000000000000..2de945286d910 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java @@ -0,0 +1,25 @@ +/* + * 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.integ.testsuite.schema; + +public class SchemaUtils { + + public static final String SOURCE_ORDERING_FIELD = "test_suite_source_ordering_field"; + +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java new file mode 100644 index 0000000000000..e67c5afae80dc --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.integ.testsuite.dag.WriterContext; +import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Appends source ordering field to both source and target schemas. This is required to assist in validation to differentiate records written in different batches. + */ +public class TestSuiteFileBasedSchemaProvider extends FilebasedSchemaProvider { + + protected static Logger log = LogManager.getLogger(WriterContext.class); + + public TestSuiteFileBasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + this.sourceSchema = addSourceOrderingFieldToSchema(sourceSchema); + this.targetSchema = addSourceOrderingFieldToSchema(targetSchema); + } + + private Schema addSourceOrderingFieldToSchema(Schema schema) { + List fields = new ArrayList<>(); + for (Schema.Field field : schema.getFields()) { + Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal()); + for (Map.Entry prop : field.getObjectProps().entrySet()) { + newField.addProp(prop.getKey(), prop.getValue()); + } + fields.add(newField); + } + Schema.Field sourceOrderingField = + new Schema.Field(SchemaUtils.SOURCE_ORDERING_FIELD, Schema.create(Type.INT), "", 0); + fields.add(sourceOrderingField); + Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false); + mergedSchema.setFields(fields); + return mergedSchema; + } + +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java index 65e4ee13cf55f..4bd096ae0cbde 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java @@ -18,6 +18,8 @@ package org.apache.hudi.integ.testsuite.writer; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + import org.apache.avro.generic.GenericRecord; import java.io.IOException; @@ -30,22 +32,29 @@ */ public class DFSDeltaWriterAdapter implements DeltaWriterAdapter { - private DeltaInputWriter deltaInputGenerator; + private DeltaInputWriter deltaInputWriter; private List metrics = new ArrayList<>(); + private int preCombineFieldVal = 0; + + public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter, int preCombineFieldVal) { + this.deltaInputWriter = deltaInputWriter; + this.preCombineFieldVal = preCombineFieldVal; + } - public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputGenerator) { - this.deltaInputGenerator = deltaInputGenerator; + public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter) { + this.deltaInputWriter = deltaInputWriter; } @Override public List write(Iterator input) throws IOException { while (input.hasNext()) { GenericRecord next = input.next(); - if (this.deltaInputGenerator.canWrite()) { - this.deltaInputGenerator.writeData(next); - } else if (input.hasNext()) { + next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal); + if (this.deltaInputWriter.canWrite()) { + this.deltaInputWriter.writeData(next); + } else { rollOver(); - this.deltaInputGenerator.writeData(next); + this.deltaInputWriter.writeData(next); } } close(); @@ -54,11 +63,11 @@ public List write(Iterator input) throws IOExcep public void rollOver() throws IOException { close(); - this.deltaInputGenerator = this.deltaInputGenerator.getNewWriter(); + this.deltaInputWriter = this.deltaInputWriter.getNewWriter(); } private void close() throws IOException { - this.deltaInputGenerator.close(); - this.metrics.add(this.deltaInputGenerator.getDeltaWriteStats()); + this.deltaInputWriter.close(); + this.metrics.add(this.deltaInputWriter.getDeltaWriteStats()); } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java index b4d9b9f8956d8..a00e8e15d401a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java @@ -18,16 +18,17 @@ package org.apache.hudi.integ.testsuite.writer; -import java.io.IOException; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; +import org.apache.avro.generic.GenericRecord; + +import java.io.IOException; + /** - * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputMode} and - * {@link DeltaInputType}. + * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputMode} and {@link DeltaInputType}. */ public class DeltaWriterFactory { @@ -44,9 +45,9 @@ public static DeltaWriterAdapter getDeltaWriterAdapter(DeltaConfig config, Integ DeltaInputWriter fileDeltaInputGenerator = new AvroFileDeltaInputWriter( dfsDeltaConfig.getConfiguration(), StringUtils - .join(new String[]{dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()}, + .join(new String[] {dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()}, "/"), dfsDeltaConfig.getSchemaStr(), dfsDeltaConfig.getMaxFileSize()); - return new DFSDeltaWriterAdapter(fileDeltaInputGenerator); + return new DFSDeltaWriterAdapter(fileDeltaInputGenerator, batchId); default: throw new IllegalArgumentException("Invalid delta input format " + config.getDeltaInputType()); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 8cb64f83e23ab..80ed1d4bf40f1 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -61,9 +61,9 @@ public abstract class ITTestBase { protected static final String HIVESERVER = "/hiveserver"; protected static final String PRESTO_COORDINATOR = "/presto-coordinator-1"; protected static final String HOODIE_WS_ROOT = "/var/hoodie/ws"; - protected static final String HOODIE_JAVA_APP = HOODIE_WS_ROOT + "/hudi-spark/run_hoodie_app.sh"; - protected static final String HOODIE_GENERATE_APP = HOODIE_WS_ROOT + "/hudi-spark/run_hoodie_generate_app.sh"; - protected static final String HOODIE_JAVA_STREAMING_APP = HOODIE_WS_ROOT + "/hudi-spark/run_hoodie_streaming_app.sh"; + protected static final String HOODIE_JAVA_APP = HOODIE_WS_ROOT + "/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh"; + protected static final String HOODIE_GENERATE_APP = HOODIE_WS_ROOT + "/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh"; + protected static final String HOODIE_JAVA_STREAMING_APP = HOODIE_WS_ROOT + "/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh"; protected static final String HUDI_HADOOP_BUNDLE = HOODIE_WS_ROOT + "/docker/hoodie/hadoop/hive_base/target/hoodie-hadoop-mr-bundle.jar"; protected static final String HUDI_HIVE_SYNC_BUNDLE = diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java index 1e5ca6886c8e6..82350999ea42c 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java @@ -18,18 +18,19 @@ package org.apache.hudi.integ.testsuite.configuration; -import static junit.framework.Assert.assertTrue; -import static junit.framework.TestCase.assertEquals; - -import java.util.ArrayList; -import java.util.List; - +import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode; import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode; -import org.apache.hudi.integ.testsuite.dag.WorkflowDag; + import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.Assert.assertTrue; +import static junit.framework.TestCase.assertEquals; + /** * Unit test for the build process of {@link DagNode} and {@link WorkflowDag}. */ diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java index d94174471bd64..70e6da7d3c49a 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java @@ -47,6 +47,9 @@ public void testConvertDagToYaml() throws Exception { public void testConvertYamlToDag() throws Exception { WorkflowDag dag = DagUtils.convertYamlToDag(UtilitiesTestBase.Helpers .readFileFromAbsolutePath((System.getProperty("user.dir") + "/.." + COW_DAG_DOCKER_DEMO_RELATIVE_PATH))); + assertEquals(dag.getDagName(), "unit-test-cow-dag"); + assertEquals(dag.getRounds(), 1); + assertEquals(dag.getIntermittentDelayMins(), 10); assertEquals(dag.getNodeList().size(), 1); Assertions.assertEquals(((DagNode) dag.getNodeList().get(0)).getParentNodes().size(), 0); assertEquals(((DagNode) dag.getNodeList().get(0)).getChildNodes().size(), 1); diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java index 94515959d01cd..2b3a65c713ea6 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java @@ -27,6 +27,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -133,4 +134,38 @@ public void testComplexPayloadWithLargeMinSize() throws Exception { assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize + 0.1 * minPayloadSize); } + @Test + public void testUpdatePayloadGeneratorWithTimestamp() throws IOException { + Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers + .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH)); + GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema); + List insertRowKeys = new ArrayList<>(); + List updateRowKeys = new ArrayList<>(); + List insertTimeStamps = new ArrayList<>(); + List updateTimeStamps = new ArrayList<>(); + List records = new ArrayList<>(); + Long startMillis = System.currentTimeMillis() - TimeUnit.MILLISECONDS + .convert(GenericRecordFullPayloadGenerator.DEFAULT_NUM_DATE_PARTITIONS, TimeUnit.DAYS); + + // Generate 10 new records + IntStream.range(0, 10).forEach(a -> { + GenericRecord record = payloadGenerator.getNewPayloadWithTimestamp("timestamp"); + records.add(record); + insertRowKeys.add(record.get("_row_key").toString()); + insertTimeStamps.add((Long) record.get("timestamp")); + }); + Set blacklistFields = new HashSet<>(Arrays.asList("_row_key")); + records.stream().forEach(a -> { + // Generate 10 updated records + GenericRecord record = payloadGenerator.getUpdatePayloadWithTimestamp(a, blacklistFields, "timestamp"); + updateRowKeys.add(record.get("_row_key").toString()); + updateTimeStamps.add((Long) record.get("timestamp")); + }); + // The row keys from insert payloads should match all the row keys from the update payloads + assertTrue(insertRowKeys.containsAll(updateRowKeys)); + // The timestamp field for the insert payloads should not all match with the update payloads + assertFalse(insertTimeStamps.containsAll(updateTimeStamps)); + Long currentMillis = System.currentTimeMillis(); + assertTrue(insertTimeStamps.stream().allMatch(t -> t >= startMillis && t <= currentMillis)); + } } diff --git a/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml b/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml index 96a6c825a98d0..23691659cab24 100644 --- a/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml +++ b/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml @@ -13,58 +13,62 @@ # 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. -first_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 2 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_insert -first_rollback: - config: - deps: second_insert - type: RollbackNode -third_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_rollback -first_upsert: - config: - record_size: 70000 - num_partitions_upsert: 1 - repeat_count: 1 - num_records_upsert: 100 - type: UpsertNode - deps: third_insert -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - hive_props: - prop2: "set spark.yarn.queue=" - prop3: "set hive.strict.checks.large.query=false" - prop4: "set hive.stats.autogather=false" - hive_queries: - query1: "select count(*) from testdb1.table1" - result1: 300 - query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" - result2: 0 - type: HiveQueryNode - deps: first_hive_sync \ No newline at end of file +dag_name: unit-test-cow-dag +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_insert + first_rollback: + config: + deps: second_insert + type: RollbackNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_rollback + first_upsert: + config: + record_size: 70000 + num_partitions_upsert: 1 + repeat_count: 1 + num_records_upsert: 100 + type: UpsertNode + deps: third_insert + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + hive_props: + prop2: "set spark.yarn.queue=" + prop3: "set hive.strict.checks.large.query=false" + prop4: "set hive.stats.autogather=false" + hive_queries: + query1: "select count(*) from testdb1.table1" + result1: 300 + query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" + result2: 0 + type: HiveQueryNode + deps: first_hive_sync \ No newline at end of file diff --git a/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml b/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml index 96a6c825a98d0..2ba42455d4874 100644 --- a/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml +++ b/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml @@ -13,58 +13,62 @@ # 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. -first_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 2 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_insert -first_rollback: - config: - deps: second_insert - type: RollbackNode -third_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_rollback -first_upsert: - config: - record_size: 70000 - num_partitions_upsert: 1 - repeat_count: 1 - num_records_upsert: 100 - type: UpsertNode - deps: third_insert -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - hive_props: - prop2: "set spark.yarn.queue=" - prop3: "set hive.strict.checks.large.query=false" - prop4: "set hive.stats.autogather=false" - hive_queries: - query1: "select count(*) from testdb1.table1" - result1: 300 - query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" - result2: 0 - type: HiveQueryNode - deps: first_hive_sync \ No newline at end of file +dag_name: unit-test-mor-dag +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_insert + first_rollback: + config: + deps: second_insert + type: RollbackNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_rollback + first_upsert: + config: + record_size: 70000 + num_partitions_upsert: 1 + repeat_count: 1 + num_records_upsert: 100 + type: UpsertNode + deps: third_insert + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + hive_props: + prop2: "set spark.yarn.queue=" + prop3: "set hive.strict.checks.large.query=false" + prop4: "set hive.stats.autogather=false" + hive_queries: + query1: "select count(*) from testdb1.table1" + result1: 300 + query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" + result2: 0 + type: HiveQueryNode + deps: first_hive_sync \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml new file mode 100644 index 0000000000000..fd887e93f2384 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/pom.xml @@ -0,0 +1,210 @@ + + + + + hudi-spark-datasource + org.apache.hudi + 0.8.0-SNAPSHOT + + 4.0.0 + + hudi-spark-common + 0.8.0-SNAPSHOT + + hudi-spark-common + jar + + + ${project.parent.parent.basedir} + + + + + + src/main/resources + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + -nobootcp + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/lib + true + true + true + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + org.jacoco + jacoco-maven-plugin + + + + + + + + org.scala-lang + scala-library + ${scala.version} + + + + + org.apache.hudi + hudi-client-common + ${project.version} + + + org.apache.hudi + hudi-spark-client + ${project.version} + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-hive-sync + ${project.version} + + + + org.apache.spark + spark-sql_${scala.binary.version} + + + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-client + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java similarity index 96% rename from hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 37572c3678485..8d3e81b0b218b 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -39,13 +39,14 @@ import org.apache.hudi.common.util.TablePathUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; -import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser; import org.apache.hudi.table.BulkInsertPartitioner; @@ -177,10 +178,12 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base } return builder.forTable(tblName) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY())) .withInlineCompaction(inlineCompact).build()) + .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY())) + .build()) // override above with Hoodie configs specified as options. .withProps(parameters).build(); } @@ -191,7 +194,7 @@ public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, Stri } public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) { - if (operation == WriteOperationType.INSERT_OVERWRITE) { + if (operation == WriteOperationType.INSERT_OVERWRITE || operation == WriteOperationType.INSERT_OVERWRITE_TABLE) { return HoodieTimeline.REPLACE_COMMIT_ACTION; } else { return CommitUtils.getCommitActionType(tableType); @@ -211,6 +214,8 @@ public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, Jav return new HoodieWriteResult(client.upsert(hoodieRecords, instantTime)); case INSERT_OVERWRITE: return client.insertOverwrite(hoodieRecords, instantTime); + case INSERT_OVERWRITE_TABLE: + return client.insertOverwriteTable(hoodieRecords, instantTime); default: throw new HoodieException("Not a valid operation type for doWriteOperation: " + operation.toString()); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java new file mode 100644 index 0000000000000..e75c9a213f36d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; + +/** + * Base class for DefaultSource used by Spark datasource v2. + */ +public class BaseDefaultSource { + + protected SparkSession sparkSession = null; + protected Configuration configuration = null; + + protected SparkSession getSparkSession() { + if (sparkSession == null) { + sparkSession = SparkSession.builder().getOrCreate(); + } + return sparkSession; + } + + protected Configuration getConfiguration() { + if (configuration == null) { + this.configuration = getSparkSession().sparkContext().hadoopConfiguration(); + } + return configuration; + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java new file mode 100644 index 0000000000000..88a7921236a58 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; + +import java.util.Arrays; +import java.util.List; + +/** + * Base class for HoodieWriterCommitMessage used by Spark datasource v2. + */ +public class BaseWriterCommitMessage { + + private List writeStatuses; + + public BaseWriterCommitMessage(List writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public List getWriteStatuses() { + return writeStatuses; + } + + @Override + public String toString() { + return "HoodieWriterCommitMessage{" + "writeStatuses=" + Arrays.toString(writeStatuses.toArray()) + '}'; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java similarity index 80% rename from hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java index 7aa0fc6a3846f..eb26c4f3209c4 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -27,8 +27,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; import org.apache.spark.sql.types.StructType; import java.io.IOException; @@ -37,12 +35,11 @@ import java.util.UUID; /** - * Hoodie's Implementation of {@link DataWriter}. This is used in data source implementation for bulk insert. + * Helper class for HoodieBulkInsertDataInternalWriter used by Spark datasource v2. */ -public class HoodieBulkInsertDataInternalWriter implements DataWriter { +public class BulkInsertDataInternalWriterHelper { - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(HoodieBulkInsertDataInternalWriter.class); + private static final Logger LOG = LogManager.getLogger(BulkInsertDataInternalWriterHelper.class); private final String instantTime; private final int taskPartitionId; @@ -55,12 +52,11 @@ public class HoodieBulkInsertDataInternalWriter implements DataWriter getWriteStatuses() throws IOException { close(); - return new HoodieWriterCommitMessage(writeStatusList); + return writeStatusList; } - @Override - public void abort() throws IOException { + public void abort() { } private void createNewHandle(String partitionPath) throws IOException { @@ -110,10 +103,11 @@ private void createNewHandle(String partitionPath) throws IOException { public void close() throws IOException { if (null != handle) { writeStatusList.add(handle.close()); + handle = null; } } - protected String getNextFileId() { + private String getNextFileId() { return String.format("%s-%d", fileIdPrefix, numFilesWritten++); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java similarity index 65% rename from hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java rename to hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index e8cbff80a2c2e..b40d36bea45e6 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -18,7 +18,6 @@ package org.apache.hudi.internal; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -33,43 +32,33 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; -import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; import org.apache.spark.sql.types.StructType; -import java.util.Arrays; import java.util.List; -import java.util.stream.Collectors; /** - * Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation - * of bulk insert. + * Helper class for HoodieDataSourceInternalWriter used by Spark datasource v2. */ -public class HoodieDataSourceInternalWriter implements DataSourceWriter { +public class DataSourceInternalWriterHelper { - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(HoodieDataSourceInternalWriter.class); + private static final Logger LOG = LogManager.getLogger(DataSourceInternalWriterHelper.class); public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time"; private final String instantTime; private final HoodieTableMetaClient metaClient; - private final HoodieWriteConfig writeConfig; - private final StructType structType; private final SparkRDDWriteClient writeClient; private final HoodieTable hoodieTable; private final WriteOperationType operationType; - public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType, - SparkSession sparkSession, Configuration configuration) { + public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession sparkSession, Configuration configuration) { this.instantTime = instantTime; - this.writeConfig = writeConfig; - this.structType = structType; this.operationType = WriteOperationType.BULK_INSERT; this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true); writeClient.setOperationType(operationType); @@ -78,32 +67,15 @@ public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writ this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient); } - @Override - public DataWriterFactory createWriterFactory() { - metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); - if (WriteOperationType.BULK_INSERT == operationType) { - return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType); - } else { - throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported "); - } - } - - @Override public boolean useCommitCoordinator() { return true; } - @Override - public void onDataWriterCommit(WriterCommitMessage message) { - LOG.info("Received commit of a data writer =" + message); + public void onDataWriterCommit(String message) { + LOG.info("Received commit of a data writer = " + message); } - @Override - public void commit(WriterCommitMessage[] messages) { - List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList()); - + public void commit(List writeStatList) { try { writeClient.commitStats(instantTime, writeStatList, Option.empty(), DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType())); @@ -114,10 +86,22 @@ public void commit(WriterCommitMessage[] messages) { } } - @Override - public void abort(WriterCommitMessage[] messages) { + public void abort() { LOG.error("Commit " + instantTime + " aborted "); writeClient.rollback(instantTime); writeClient.close(); } + + public void createInflightCommit() { + metaClient.getActiveTimeline().transitionRequestedToInflight( + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); + } + + public HoodieTable getHoodieTable() { + return hoodieTable; + } + + public WriteOperationType getWriteOperationType() { + return operationType; + } } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala similarity index 96% rename from hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 73f70e72d3bb6..f0974977e4a4b 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -20,6 +20,7 @@ package org.apache.hudi import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload import org.apache.hudi.common.model.WriteOperationType +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hive.HiveSyncTool import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor import org.apache.hudi.keygen.SimpleKeyGenerator @@ -36,7 +37,7 @@ import org.apache.log4j.LogManager */ object DataSourceReadOptions { - private val log = LogManager.getLogger(classOf[DefaultSource]) + private val log = LogManager.getLogger(DataSourceReadOptions.getClass) /** * Whether data needs to be read, in @@ -143,7 +144,7 @@ object DataSourceReadOptions { */ object DataSourceWriteOptions { - private val log = LogManager.getLogger(classOf[DefaultSource]) + private val log = LogManager.getLogger(DataSourceWriteOptions.getClass) /** * The write operation, that this write should do @@ -157,6 +158,7 @@ object DataSourceWriteOptions { val DELETE_OPERATION_OPT_VAL = WriteOperationType.DELETE.value val BOOTSTRAP_OPERATION_OPT_VAL = WriteOperationType.BOOTSTRAP.value val INSERT_OVERWRITE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE.value + val INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE_TABLE.value val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** @@ -201,15 +203,14 @@ object DataSourceWriteOptions { * key value, we will pick the one with the largest value for the precombine field, * determined by Object.compareTo(..) */ - val PRECOMBINE_FIELD_OPT_KEY = "hoodie.datasource.write.precombine.field" + val PRECOMBINE_FIELD_OPT_KEY = HoodieWriteConfig.PRECOMBINE_FIELD_PROP val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts" - /** * Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. * This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective */ - val PAYLOAD_CLASS_OPT_KEY = "hoodie.datasource.write.payload.class" + val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName /** @@ -223,7 +224,7 @@ object DataSourceWriteOptions { /** * Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`. Actual - * value ontained by invoking .toString() + * value obtained by invoking .toString() */ val PARTITIONPATH_FIELD_OPT_KEY = KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = "partitionpath" @@ -241,7 +242,7 @@ object DataSourceWriteOptions { * Key generator class, that implements will extract the key out of incoming record * */ - val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class" + val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS_PROP val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java new file mode 100644 index 0000000000000..d66a5ee51a0c6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java @@ -0,0 +1,112 @@ +/* + * 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.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +import java.util.List; +import java.util.Random; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Base class for TestHoodieBulkInsertDataInternalWriter. + */ +public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarness { + + protected static final Random RANDOM = new Random(); + + @BeforeEach + public void setUp() throws Exception { + initSparkContexts(); + initPath(); + initFileSystem(); + initTestDataGenerator(); + initMetaClient(); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + protected void assertWriteStatuses(List writeStatuses, int batches, int size, + Option> fileAbsPaths, Option> fileNames) { + assertEquals(batches, writeStatuses.size()); + int counter = 0; + for (HoodieInternalWriteStatus writeStatus : writeStatuses) { + // verify write status + assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath()); + assertEquals(writeStatus.getTotalRecords(), size); + assertNull(writeStatus.getGlobalError()); + assertEquals(writeStatus.getFailedRowsSize(), 0); + assertEquals(writeStatus.getTotalErrorRecords(), 0); + assertFalse(writeStatus.hasErrors()); + assertNotNull(writeStatus.getFileId()); + String fileId = writeStatus.getFileId(); + if (fileAbsPaths.isPresent()) { + fileAbsPaths.get().add(basePath + "/" + writeStatus.getStat().getPath()); + } + if (fileNames.isPresent()) { + fileNames.get().add(writeStatus.getStat().getPath() + .substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1)); + } + HoodieWriteStat writeStat = writeStatus.getStat(); + assertEquals(size, writeStat.getNumInserts()); + assertEquals(size, writeStat.getNumWrites()); + assertEquals(fileId, writeStat.getFileId()); + assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); + assertEquals(0, writeStat.getNumDeletes()); + assertEquals(0, writeStat.getNumUpdateWrites()); + assertEquals(0, writeStat.getTotalWriteErrors()); + } + } + + protected void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime, Option> fileNames) { + // verify 3 meta fields that are filled in within create handle + actualRows.collectAsList().forEach(entry -> { + assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); + assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); + if (fileNames.isPresent()) { + assertTrue(fileNames.get().contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS + .get(HoodieRecord.FILENAME_METADATA_FIELD)))); + } + assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); + }); + + // after trimming 2 of the meta fields, rest of the fields should match + Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + assertEquals(0, trimmedActual.except(trimmedExpected).count()); + } +} diff --git a/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml similarity index 94% rename from hudi-spark/pom.xml rename to hudi-spark-datasource/hudi-spark/pom.xml index 0942327542a19..4f56c7e9391b7 100644 --- a/hudi-spark/pom.xml +++ b/hudi-spark-datasource/hudi-spark/pom.xml @@ -17,17 +17,20 @@ --> - hudi + hudi-spark-datasource org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 hudi-spark_${scala.binary.version} + 0.8.0-SNAPSHOT + + hudi-spark_${scala.binary.version} jar - ${project.parent.basedir} + ${project.parent.parent.basedir} @@ -196,6 +199,21 @@ hudi-sync-common ${project.version} + + org.apache.hudi + hudi-spark-common + ${project.version} + + + org.apache.hudi + hudi-spark2_${scala.binary.version} + ${project.version} + + + org.apache.hudi + hudi-spark3_2.12 + ${project.version} + diff --git a/hudi-spark/run_hoodie_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh similarity index 93% rename from hudi-spark/run_hoodie_app.sh rename to hudi-spark-datasource/hudi-spark/run_hoodie_app.sh index 7c63e7411eb48..9782aa359556f 100755 --- a/hudi-spark/run_hoodie_app.sh +++ b/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh @@ -23,7 +23,7 @@ function error_exit { DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" #Ensure we pick the right jar even for hive11 builds -HUDI_JAR=`ls -c $DIR/../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v source | head -1` +HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1` if [ -z "$HADOOP_CONF_DIR" ]; then echo "setting hadoop conf dir" diff --git a/hudi-spark/run_hoodie_generate_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh similarity index 93% rename from hudi-spark/run_hoodie_generate_app.sh rename to hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh index a4b4090b049a8..a2769517b9eb4 100755 --- a/hudi-spark/run_hoodie_generate_app.sh +++ b/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh @@ -23,7 +23,7 @@ function error_exit { DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" #Ensure we pick the right jar even for hive11 builds -HUDI_JAR=`ls -c $DIR/../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v source | head -1` +HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1` if [ -z "$HADOOP_CONF_DIR" ]; then echo "setting hadoop conf dir" diff --git a/hudi-spark/run_hoodie_streaming_app.sh b/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh similarity index 94% rename from hudi-spark/run_hoodie_streaming_app.sh rename to hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh index 01f1a4e4a13f0..9a81a4c0684e3 100755 --- a/hudi-spark/run_hoodie_streaming_app.sh +++ b/hudi-spark-datasource/hudi-spark/run_hoodie_streaming_app.sh @@ -23,7 +23,7 @@ function error_exit { DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" #Ensure we pick the right jar even for hive11 builds -HUDI_JAR=`ls -c $DIR/../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v source | head -1` +HUDI_JAR=`ls -c $DIR/../../packaging/hudi-spark-bundle/target/hudi-spark-bundle*.jar | grep -v sources | head -1` if [ -z "$HADOOP_CONF_DIR" ]; then echo "setting hadoop conf dir" diff --git a/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java similarity index 96% rename from hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java index 1467cf6bca8f0..734e0c0ea7a91 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java @@ -74,7 +74,8 @@ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, Strin if (metaClient.getTableType().equals(HoodieTableType.MERGE_ON_READ)) { return metaClient.getActiveTimeline().getTimelineOfActions( CollectionUtils.createSet(HoodieActiveTimeline.COMMIT_ACTION, - HoodieActiveTimeline.DELTA_COMMIT_ACTION)).filterCompletedInstants(); + HoodieActiveTimeline.DELTA_COMMIT_ACTION, + HoodieActiveTimeline.REPLACE_COMMIT_ACTION)).filterCompletedInstants(); } else { return metaClient.getCommitTimeline().filterCompletedInstants(); } diff --git a/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java similarity index 100% rename from hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java diff --git a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java similarity index 89% rename from hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java index febdf190c40b4..2ba1e286481cc 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java @@ -121,11 +121,22 @@ public static GenericRecord generateGenericRecord(String rowKey, String riderNam */ public static OverwriteWithLatestAvroPayload generateRandomValue(HoodieKey key, String riderDriverSuffix) throws IOException { + // The timestamp generated is limited to range from 7 days before to now, to avoid generating too many + // partitionPaths when user use timestamp as partitionPath filed. GenericRecord rec = - generateGenericRecord(key.getRecordKey(), "rider-" + riderDriverSuffix, "driver-" + riderDriverSuffix, 0); + generateGenericRecord(key.getRecordKey(), "rider-" + riderDriverSuffix, "driver-" + + riderDriverSuffix, generateRangeRandomTimestamp(7)); return new OverwriteWithLatestAvroPayload(Option.of(rec)); } + /** + * Generate timestamp range from {@param daysTillNow} before to now. + */ + private static long generateRangeRandomTimestamp(int daysTillNow) { + long maxIntervalMillis = daysTillNow * 24 * 60 * 60 * 1000L; + return System.currentTimeMillis() - (long)(Math.random() * maxIntervalMillis); + } + /** * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys. */ @@ -182,8 +193,12 @@ public List generateUpdates(Integer n) throws IOException { * @return list of hoodie records to delete */ public List generateDeletes(List rows) { - return rows.stream().map(row -> - convertToString(row.getAs("uuid"), row.getAs("partitionpath"))).filter(os -> os.isPresent()).map(os -> os.get()) + // if row.length() == 2, then the record contains "uuid" and "partitionpath" fields, otherwise, + // another field "ts" is available + return rows.stream().map(row -> row.length() == 2 + ? convertToString(row.getAs("uuid"), row.getAs("partitionpath"), null) : + convertToString(row.getAs("uuid"), row.getAs("partitionpath"), row.getAs("ts")) + ).filter(os -> os.isPresent()).map(os -> os.get()) .collect(Collectors.toList()); } @@ -204,10 +219,10 @@ private static Option convertToString(HoodieRecord record) { } } - private static Option convertToString(String uuid, String partitionPath) { + private static Option convertToString(String uuid, String partitionPath, Long ts) { StringBuffer stringBuffer = new StringBuffer(); stringBuffer.append("{"); - stringBuffer.append("\"ts\": 0.0,"); + stringBuffer.append("\"ts\": \"" + (ts == null ? "0.0" : ts) + "\","); stringBuffer.append("\"uuid\": \"" + uuid + "\","); stringBuffer.append("\"partitionpath\": \"" + partitionPath + "\""); stringBuffer.append("}"); diff --git a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java similarity index 92% rename from hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java index d1a415b175200..130ea7c27595a 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java @@ -21,7 +21,7 @@ import org.apache.hudi.client.AbstractCompactor; import org.apache.hudi.client.AbstractHoodieWriteClient; import org.apache.hudi.client.HoodieSparkCompactor; -import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; /** * Async Compaction Service used by Structured Streaming. Here, async compaction is run in daemon mode to prevent @@ -37,6 +37,6 @@ public SparkStreamingAsyncCompactService(HoodieEngineContext context, AbstractHo @Override protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) { - return new HoodieSparkCompactor(client); + return new HoodieSparkCompactor(client, this.context); } } diff --git a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java similarity index 95% rename from hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java index 022abe3085de6..6c5eb0ed5748d 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java @@ -18,8 +18,8 @@ package org.apache.hudi.bootstrap; -import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; @@ -65,7 +65,7 @@ public JavaRDD generateInputRecords(String tableName, String sourc KeyGenerator keyGenerator = DataSourceUtils.createKeyGenerator(props); String structName = tableName + "_record"; String namespace = "hoodie." + tableName; - RDD genericRecords = AvroConversionUtils.createRdd(inputDataset, structName, namespace); + RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace); return genericRecords.toJavaRDD().map(gr -> { String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( gr, props.getString("hoodie.datasource.write.precombine.field"), false); diff --git a/hudi-spark/src/main/java/org/apache/hudi/exception/HoodieDeltaStreamerException.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/exception/HoodieDeltaStreamerException.java similarity index 100% rename from hudi-spark/src/main/java/org/apache/hudi/exception/HoodieDeltaStreamerException.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/exception/HoodieDeltaStreamerException.java diff --git a/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java similarity index 100% rename from hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java rename to hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java diff --git a/hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/hudi-spark-datasource/hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister similarity index 100% rename from hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister rename to hudi-spark-datasource/hudi-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala similarity index 92% rename from hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index 4a7837816976b..d26390d635a6e 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -19,7 +19,7 @@ package org.apache.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceReadOptions._ -import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION_OPT_KEY} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.table.HoodieTableMetaClient @@ -33,6 +33,8 @@ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import scala.collection.JavaConverters._ + /** * Hoodie Spark Datasource, for reading and writing hoodie tables * @@ -92,7 +94,12 @@ class DefaultSource extends RelationProvider } else if(parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)) { getBaseFileOnlyView(sqlContext, parameters, schema, readPaths, isBootstrappedTable, globPaths, metaClient) } else if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_INCREMENTAL_OPT_VAL)) { - new IncrementalRelation(sqlContext, tablePath, optParams, schema) + val metaClient = new HoodieTableMetaClient(fs.getConf, tablePath) + if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { + new MergeOnReadIncrementalRelation(sqlContext, optParams, schema, metaClient) + } else { + new IncrementalRelation(sqlContext, optParams, schema, metaClient) + } } else { throw new HoodieException("Invalid query type :" + parameters(QUERY_TYPE_OPT_KEY)) } @@ -119,12 +126,14 @@ class DefaultSource extends RelationProvider optParams: Map[String, String], df: DataFrame): BaseRelation = { val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams) + val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) + if (parameters(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) { - HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, df) + HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, dfWithoutMetaCols) } else { - HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df) + HoodieSparkSqlWriter.write(sqlContext, mode, parameters, dfWithoutMetaCols) } - new HoodieEmptyRelation(sqlContext, df.schema) + new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema) } override def createSink(sqlContext: SQLContext, diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala similarity index 100% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRDD.scala diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala similarity index 95% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala index a1e9947cafacf..f7415f93f6b2d 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala @@ -91,6 +91,9 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, // Get required schemas for column pruning var requiredDataSchema = StructType(Seq()) var requiredSkeletonSchema = StructType(Seq()) + // requiredColsSchema is the schema of requiredColumns, note that requiredColumns is in a random order + // so requiredColsSchema is not always equal to (requiredSkeletonSchema.fields ++ requiredDataSchema.fields) + var requiredColsSchema = StructType(Seq()) requiredColumns.foreach(col => { var field = dataSchema.find(_.name == col) if (field.isDefined) { @@ -99,6 +102,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, field = skeletonSchema.find(_.name == col) requiredSkeletonSchema = requiredSkeletonSchema.add(field.get) } + requiredColsSchema = requiredColsSchema.add(field.get) }) // Prepare readers for reading data file and skeleton files @@ -129,7 +133,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, sparkSession = _sqlContext.sparkSession, dataSchema = fullSchema, partitionSchema = StructType(Seq.empty), - requiredSchema = StructType(requiredSkeletonSchema.fields ++ requiredDataSchema.fields), + requiredSchema = requiredColsSchema, filters = filters, options = Map.empty, hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf()) diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala similarity index 100% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieEmptyRelation.scala diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala similarity index 74% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 148102480c3c5..e20c33c28e883 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -50,30 +50,32 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, private val confBroadcast = sc.broadcast(new SerializableWritable(config)) override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val mergeParquetPartition = split.asInstanceOf[HoodieMergeOnReadPartition] - mergeParquetPartition.split match { + val mergeOnReadPartition = split.asInstanceOf[HoodieMergeOnReadPartition] + mergeOnReadPartition.split match { case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty => - read(mergeParquetPartition.split.dataFile, requiredSchemaFileReader) + read(dataFileOnlySplit.dataFile.get, requiredSchemaFileReader) + case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty => + logFileIterator(logFileOnlySplit, getConfig) case skipMergeSplit if skipMergeSplit.mergeType .equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) => skipMergeFileIterator( skipMergeSplit, - read(mergeParquetPartition.split.dataFile, requiredSchemaFileReader), + read(skipMergeSplit.dataFile.get, requiredSchemaFileReader), getConfig ) case payloadCombineSplit if payloadCombineSplit.mergeType .equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) => payloadCombineFileIterator( payloadCombineSplit, - read(mergeParquetPartition.split.dataFile, fullSchemaFileReader), + read(payloadCombineSplit.dataFile.get, fullSchemaFileReader), getConfig ) case _ => throw new HoodieException(s"Unable to select an Iterator to read the Hoodie MOR File Split for " + - s"file path: ${mergeParquetPartition.split.dataFile.filePath}" + - s"log paths: ${mergeParquetPartition.split.logPaths.toString}" + - s"hoodie table path: ${mergeParquetPartition.split.tablePath}" + - s"spark partition Index: ${mergeParquetPartition.index}" + - s"merge type: ${mergeParquetPartition.split.mergeType}") + s"file path: ${mergeOnReadPartition.split.dataFile.get.filePath}" + + s"log paths: ${mergeOnReadPartition.split.logPaths.toString}" + + s"hoodie table path: ${mergeOnReadPartition.split.tablePath}" + + s"spark partition Index: ${mergeOnReadPartition.index}" + + s"merge type: ${mergeOnReadPartition.split.mergeType}") } } @@ -101,6 +103,44 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, rows } + private def logFileIterator(split: HoodieMergeOnReadFileSplit, + config: Configuration): Iterator[InternalRow] = + new Iterator[InternalRow] { + private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema) + private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema) + private val requiredFieldPosition = + tableState.requiredStructSchema + .map(f => tableAvroSchema.getField(f.name).pos()).toList + private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema) + private val deserializer = new AvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema) + private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema) + private val logRecords = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config).getRecords + private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala + + private var recordToLoad: InternalRow = _ + override def hasNext: Boolean = { + if (logRecordsKeyIterator.hasNext) { + val curAvrokey = logRecordsKeyIterator.next() + val curAvroRecord = logRecords.get(curAvrokey).getData.getInsertValue(tableAvroSchema) + if (!curAvroRecord.isPresent) { + // delete record found, skipping + this.hasNext + } else { + val requiredAvroRecord = AvroConversionUtils + .buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder) + recordToLoad = unsafeProjection(deserializer.deserialize(requiredAvroRecord).asInstanceOf[InternalRow]) + true + } + } else { + false + } + } + + override def next(): InternalRow = { + recordToLoad + } + } + private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit, baseFileIterator: Iterator[InternalRow], config: Configuration): Iterator[InternalRow] = @@ -255,19 +295,24 @@ private object HoodieMergeOnReadRDD { def scanLog(split: HoodieMergeOnReadFileSplit, logSchema: Schema, config: Configuration): HoodieMergedLogRecordScanner = { val fs = FSUtils.getFs(split.tablePath, config) - new HoodieMergedLogRecordScanner( - fs, - split.tablePath, - split.logPaths.get.asJava, - logSchema, - split.latestCommit, - split.maxCompactionMemoryInBytes, - Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, - HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean).getOrElse(false), - false, - config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, - HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), - config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, - HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(split.tablePath) + .withLogFilePaths(split.logPaths.get.asJava) + .withReaderSchema(logSchema) + .withLatestInstantTime(split.latestCommit) + .withReadBlocksLazily( + Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, + HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean) + .getOrElse(false)) + .withReverseReader(false) + .withBufferSize( + config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, + HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)) + .withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes) + .withSpillableMapBasePath( + config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP, + HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)) + .build() } } diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala similarity index 90% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index c7d5c8782db7b..2b3a3498903ff 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -29,7 +29,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.HoodieWriteResult import org.apache.hudi.client.SparkRDDWriteClient -import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.HoodieActiveTimeline @@ -38,9 +38,10 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} -import org.apache.hudi.internal.HoodieDataSourceInternalWriter +import org.apache.hudi.internal.{DataSourceInternalWriterHelper, HoodieDataSourceInternalWriter} import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.log4j.LogManager +import org.apache.spark.SPARK_VERSION import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD @@ -120,7 +121,8 @@ private[hudi] object HoodieSparkSqlWriter { // short-circuit if bulk_insert via row is enabled. // scalastyle:off - if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean) { + if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean && + operation == WriteOperationType.BULK_INSERT) { val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName, basePath, path, instantTime) return (success, commitTime, common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig) @@ -140,7 +142,7 @@ private[hudi] object HoodieSparkSqlWriter { // Convert to RDD[HoodieRecord] val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters)) - val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, schema, structName, nameSpace) + val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, schema, structName, nameSpace) val shouldCombine = parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean || operation.equals(WriteOperationType.UPSERT); val hoodieAllIncomingRecords = genericRecords.map(gr => { val hoodieRecord = if (shouldCombine) { @@ -157,7 +159,7 @@ private[hudi] object HoodieSparkSqlWriter { // Create a HoodieWriteClient & issue the write. val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get, - tblName, mapAsJavaMap(parameters) + tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP) )).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) { @@ -187,7 +189,7 @@ private[hudi] object HoodieSparkSqlWriter { // Convert to RDD[HoodieKey] val keyGenerator = DataSourceUtils.createKeyGenerator(toProperties(parameters)) - val genericRecords: RDD[GenericRecord] = AvroConversionUtils.createRdd(df, structName, nameSpace) + val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, structName, nameSpace) val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD() if (!tableExists) { @@ -197,7 +199,8 @@ private[hudi] object HoodieSparkSqlWriter { // Create a HoodieWriteClient & issue the delete. val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, Schema.create(Schema.Type.NULL).toString, path.get, tblName, - mapAsJavaMap(parameters))).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] + mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP))) + .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) { asyncCompactionTriggerFn.get.apply(client) @@ -286,10 +289,22 @@ private[hudi] object HoodieSparkSqlWriter { val nameSpace = s"hoodie.${tblName}" val writeConfig = DataSourceUtils.createHoodieConfig(null, path.get, tblName, mapAsJavaMap(parameters)) val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace) - hoodieDF.write.format("org.apache.hudi.internal") - .option(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY, instantTime) - .options(parameters) - .save() + if (SPARK_VERSION.startsWith("2.")) { + hoodieDF.write.format("org.apache.hudi.internal") + .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) + .options(parameters) + .save() + } else if (SPARK_VERSION.startsWith("3.")) { + hoodieDF.write.format("org.apache.hudi.spark3.internal") + .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) + .option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL, hoodieDF.schema.toDDL) + .options(parameters) + .mode(SaveMode.Append) + .save() + } else { + throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + + " To use row writer please switch to spark 2 or spark 3") + } val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) val metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) val syncHiveSucess = if (hiveSyncEnabled || metaSyncEnabled) { @@ -318,7 +333,9 @@ private[hudi] object HoodieSparkSqlWriter { if (operation != WriteOperationType.DELETE) { if (mode == SaveMode.ErrorIfExists && tableExists) { throw new HoodieException(s"hoodie table at $tablePath already exists.") - } else if (mode == SaveMode.Overwrite && tableExists) { + } else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) { + // When user set operation as INSERT_OVERWRITE_TABLE, + // overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation log.warn(s"hoodie table at $tablePath already exists. Deleting existing data & overwriting with new data.") fs.delete(tablePath, true) tableExists = false @@ -354,7 +371,12 @@ private[hudi] object HoodieSparkSqlWriter { ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*) hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) hiveSyncConfig.useJdbc = parameters(HIVE_USE_JDBC_OPT_KEY).toBoolean + hiveSyncConfig.useFileListingFromMetadata = parameters(HoodieMetadataConfig.METADATA_ENABLE_PROP).toBoolean + hiveSyncConfig.verifyMetadataFileListing = parameters(HoodieMetadataConfig.METADATA_VALIDATE_PROP).toBoolean hiveSyncConfig.supportTimestamp = parameters.get(HIVE_SUPPORT_TIMESTAMP).exists(r => r.toBoolean) + hiveSyncConfig.autoCreateDatabase = parameters.get(HIVE_AUTO_CREATE_DATABASE_OPT_KEY).exists(r => r.toBoolean) + hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY, + DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean hiveSyncConfig } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala new file mode 100644 index 0000000000000..02880f22b93fc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -0,0 +1,121 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.client.utils.SparkRowDeserializer +import org.apache.hudi.common.model.HoodieRecord +import org.apache.spark.SPARK_VERSION +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.avro.SchemaConverters +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +import scala.collection.JavaConverters._ + + +object HoodieSparkUtils { + + def getMetaSchema: StructType = { + StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => { + StructField(col, StringType, nullable = true) + })) + } + + /** + * This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]]. + * [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally. + */ + def isGlobPath(pattern: Path): Boolean = { + pattern.toString.exists("{}[]*?\\".toSet.contains) + } + + /** + * This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]]. + * [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally. + */ + def globPath(fs: FileSystem, pattern: Path): Seq[Path] = { + Option(fs.globStatus(pattern)).map { statuses => + statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq + }.getOrElse(Seq.empty[Path]) + } + + /** + * This method copied from [[org.apache.spark.deploy.SparkHadoopUtil]]. + * [[org.apache.spark.deploy.SparkHadoopUtil]] becomes private since Spark 3.0.0 and hence we had to copy it locally. + */ + def globPathIfNecessary(fs: FileSystem, pattern: Path): Seq[Path] = { + if (isGlobPath(pattern)) globPath(fs, pattern) else Seq(pattern) + } + + /** + * Checks to see whether input path contains a glob pattern and if yes, maps it to a list of absolute paths + * which match the glob pattern. Otherwise, returns original path + * + * @param paths List of absolute or globbed paths + * @param fs File system + * @return list of absolute file paths + */ + def checkAndGlobPathIfNecessary(paths: Seq[String], fs: FileSystem): Seq[Path] = { + paths.flatMap(path => { + val qualified = new Path(path).makeQualified(fs.getUri, fs.getWorkingDirectory) + val globPaths = globPathIfNecessary(fs, qualified) + globPaths + }) + } + + def createInMemoryFileIndex(sparkSession: SparkSession, globbedPaths: Seq[Path]): InMemoryFileIndex = { + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache) + } + + def createRdd(df: DataFrame, structName: String, recordNamespace: String): RDD[GenericRecord] = { + val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, recordNamespace) + createRdd(df, avroSchema, structName, recordNamespace) + } + + def createRdd(df: DataFrame, avroSchema: Schema, structName: String, recordNamespace: String) + : RDD[GenericRecord] = { + // Use the Avro schema to derive the StructType which has the correct nullability information + val dataType = SchemaConverters.toSqlType(avroSchema).dataType.asInstanceOf[StructType] + val encoder = RowEncoder.apply(dataType).resolveAndBind() + val deserializer = HoodieSparkUtils.createDeserializer(encoder) + df.queryExecution.toRdd.map(row => deserializer.deserializeRow(row)) + .mapPartitions { records => + if (records.isEmpty) Iterator.empty + else { + val convertor = AvroConversionHelper.createConverterToAvro(dataType, structName, recordNamespace) + records.map { x => convertor(x).asInstanceOf[GenericRecord] } + } + } + } + + def createDeserializer(encoder: ExpressionEncoder[Row]): SparkRowDeserializer = { + // TODO remove Spark2RowDeserializer if Spark 2.x support is dropped + if (SPARK_VERSION.startsWith("2.")) { + new Spark2RowDeserializer(encoder) + } else { + new Spark3RowDeserializer(encoder) + } + } +} diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala similarity index 100% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala similarity index 88% rename from hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 294050b6d46ca..02b5abd12c62c 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -23,6 +23,11 @@ import org.apache.hudi.common.config.TypedProperties import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters.mapAsScalaMapConverter +import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE +import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE +import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP +import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP + /** * WriterUtils to assist in write path in Datasource and tests. */ @@ -46,6 +51,8 @@ object HoodieWriterUtils { RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL, PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL, KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL, + METADATA_ENABLE_PROP -> DEFAULT_METADATA_ENABLE.toString, + METADATA_VALIDATE_PROP -> DEFAULT_METADATA_VALIDATE.toString, COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL, INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL, STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL, diff --git a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala similarity index 89% rename from hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala index f8345780b1c9f..5c20656c3ecd5 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala @@ -42,19 +42,14 @@ import scala.collection.mutable * */ class IncrementalRelation(val sqlContext: SQLContext, - val basePath: String, val optParams: Map[String, String], - val userSchema: StructType) extends BaseRelation with TableScan { + val userSchema: StructType, + val metaClient: HoodieTableMetaClient) extends BaseRelation with TableScan { private val log = LogManager.getLogger(classOf[IncrementalRelation]) val skeletonSchema: StructType = HoodieSparkUtils.getMetaSchema - private val metaClient = new HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration, basePath, true) - - // MOR tables not supported yet - if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) { - throw new HoodieException("Incremental view not implemented yet, for merge-on-read tables") - } + private val basePath = metaClient.getBasePath // TODO : Figure out a valid HoodieWriteConfig private val hoodieTable = HoodieSparkTable.create(HoodieWriteConfig.newBuilder().withPath(basePath).build(), new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)), @@ -92,17 +87,8 @@ class IncrementalRelation(val sqlContext: SQLContext, StructType(skeletonSchema.fields ++ dataSchema.fields) } - - private val filters = { - if (optParams.contains(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY)) { - val filterStr = optParams.getOrElse( - DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY, - DataSourceReadOptions.DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL) - filterStr.split(",").filter(!_.isEmpty) - } else { - Array[String]() - } - } + private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY, + DataSourceReadOptions.DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL).split(",").filter(!_.isEmpty) override def schema: StructType = usedSchema diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala new file mode 100644 index 0000000000000..c85b972910c54 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -0,0 +1,215 @@ +/* + * 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 + +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.exception.HoodieException +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listAffectedFilesForCommits +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes + +import org.apache.hadoop.fs.{FileStatus, GlobPattern, Path} +import org.apache.hadoop.mapred.JobConf +import org.apache.log4j.LogManager +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SQLContext} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ListBuffer + +/** + * Experimental. + * Relation, that implements the Hoodie incremental view for Merge On Read table. + * + */ +class MergeOnReadIncrementalRelation(val sqlContext: SQLContext, + val optParams: Map[String, String], + val userSchema: StructType, + val metaClient: HoodieTableMetaClient) + extends BaseRelation with PrunedFilteredScan { + + private val log = LogManager.getLogger(classOf[MergeOnReadIncrementalRelation]) + private val conf = sqlContext.sparkContext.hadoopConfiguration + private val jobConf = new JobConf(conf) + private val fs = FSUtils.getFs(metaClient.getBasePath, conf) + private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() + if (commitTimeline.empty()) { + throw new HoodieException("No instants to incrementally pull") + } + if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) { + throw new HoodieException(s"Specify the begin instant time to pull from using " + + s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}") + } + + private val lastInstant = commitTimeline.lastInstant().get() + private val mergeType = optParams.getOrElse( + DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, + DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL) + + private val commitsTimelineToReturn = commitTimeline.findInstantsInRange( + optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY), + optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp)) + log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}") + private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList + private val schemaUtil = new TableSchemaResolver(metaClient) + private val tableAvroSchema = schemaUtil.getTableAvroSchema + private val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) + private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf) + private val fileIndex = buildFileIndex() + + override def schema: StructType = tableStructSchema + + override def needConversion: Boolean = false + + override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { + val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) + val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp) + val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp) + filters :+isNotNullFilter :+ largerThanFilter :+ lessThanFilter + } + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + log.debug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}") + log.debug(s"buildScan filters = ${filters.mkString(",")}") + // config to ensure the push down filter for parquet will be applied. + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true") + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true") + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false") + val pushDownFilter = { + val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) + val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp) + val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp) + filters :+isNotNullFilter :+ largerThanFilter :+ lessThanFilter + } + var requiredStructSchema = StructType(Seq()) + requiredColumns.foreach(col => { + val field = tableStructSchema.find(_.name == col) + if (field.isDefined) { + requiredStructSchema = requiredStructSchema.add(field.get) + } + }) + val requiredAvroSchema = AvroConversionUtils + .convertStructTypeToAvroSchema(requiredStructSchema, tableAvroSchema.getName, tableAvroSchema.getNamespace) + val hoodieTableState = HoodieMergeOnReadTableState( + tableStructSchema, + requiredStructSchema, + tableAvroSchema.toString, + requiredAvroSchema.toString, + fileIndex + ) + val fullSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues( + sparkSession = sqlContext.sparkSession, + dataSchema = tableStructSchema, + partitionSchema = StructType(Nil), + requiredSchema = tableStructSchema, + filters = pushDownFilter, + options = optParams, + hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + ) + val requiredSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues( + sparkSession = sqlContext.sparkSession, + dataSchema = tableStructSchema, + partitionSchema = StructType(Nil), + requiredSchema = requiredStructSchema, + filters = pushDownFilter, + options = optParams, + hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() + ) + + val rdd = new HoodieMergeOnReadRDD( + sqlContext.sparkContext, + jobConf, + fullSchemaParquetReader, + requiredSchemaParquetReader, + hoodieTableState + ) + rdd.asInstanceOf[RDD[Row]] + } + + def buildFileIndex(): List[HoodieMergeOnReadFileSplit] = { + val partitionsWithFileStatus = listAffectedFilesForCommits(new Path(metaClient.getBasePath), + commitsToReturn, commitsTimelineToReturn) + val affectedFileStatus = new ListBuffer[FileStatus] + partitionsWithFileStatus.iterator.foreach(p => + p._2.iterator.foreach(status => affectedFileStatus += status._2)) + val fsView = new HoodieTableFileSystemView(metaClient, + commitsTimelineToReturn, affectedFileStatus.toArray) + + // Iterate partitions to create splits + val fileGroup = partitionsWithFileStatus.keySet().flatMap(partitionPath => + fsView.getAllFileGroups(partitionPath).iterator() + ).toList + val latestCommit = fsView.getLastInstant.get().getTimestamp + if (log.isDebugEnabled) { + fileGroup.foreach(f => log.debug(s"current file group id: " + + s"${f.getFileGroupId} and file slices ${f.getLatestFileSlice.get().toString}")) + } + + // Filter files based on user defined glob pattern + val pathGlobPattern = optParams.getOrElse( + DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, + DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL) + val filteredFileGroup = if(!pathGlobPattern + .equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) { + val globMatcher = new GlobPattern("*" + pathGlobPattern) + fileGroup.filter(f => { + if (f.getLatestFileSlice.get().getBaseFile.isPresent) { + globMatcher.matches(f.getLatestFileSlice.get().getBaseFile.get.getPath) + } else { + globMatcher.matches(f.getLatestFileSlice.get().getLatestLogFile.get().getPath.toString) + } + }) + } else { + fileGroup + } + + // Build HoodieMergeOnReadFileSplit. + filteredFileGroup.map(f => { + // Ensure get the base file when there is a pending compaction, which means the base file + // won't be in the latest file slice. + val baseFiles = f.getAllFileSlices.iterator().filter(slice => slice.getBaseFile.isPresent).toList + val partitionedFile = if (baseFiles.nonEmpty) { + val baseFile = baseFiles.head.getBaseFile + Option(PartitionedFile(InternalRow.empty, baseFile.get.getPath, 0, baseFile.get.getFileLen)) + } + else { + Option.empty + } + + val logPath = if (f.getLatestFileSlice.isPresent) { + //If log path doesn't exist, we still include an empty path to avoid using + // the default parquet reader to ensure the push down filter will be applied. + Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList + .map(logfile => logfile.getPath.toString)) + } + else { + Option.empty + } + + HoodieMergeOnReadFileSplit(partitionedFile, logPath, + latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) + }) + } +} diff --git a/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala similarity index 95% rename from hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index c1a6acdb04253..328e3c3cfeb95 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.StructType import scala.collection.JavaConverters._ -case class HoodieMergeOnReadFileSplit(dataFile: PartitionedFile, +case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile], logPaths: Option[List[String]], latestCommit: String, tablePath: String, @@ -99,7 +99,7 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, dataSchema = tableStructSchema, partitionSchema = StructType(Nil), requiredSchema = tableStructSchema, - filters = Seq(), + filters = filters, options = optParams, hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() ) @@ -113,9 +113,6 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf() ) - // Follow the implementation of Spark internal HadoopRDD to handle the broadcast configuration. - FileSystem.getLocal(jobConf) - SparkHadoopUtil.get.addCredentials(jobConf) val rdd = new HoodieMergeOnReadRDD( sqlContext.sparkContext, jobConf, @@ -143,7 +140,7 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext, val baseFile = kv._1 val logPaths = if (kv._2.isEmpty) Option.empty else Option(kv._2.asScala.toList) val partitionedFile = PartitionedFile(InternalRow.empty, baseFile.getPath, 0, baseFile.getFileLen) - HoodieMergeOnReadFileSplit(partitionedFile, logPaths, latestCommit, + HoodieMergeOnReadFileSplit(Option(partitionedFile), logPaths, latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) }).toList fileSplits diff --git a/hudi-spark/src/main/scala/org/apache/hudi/package.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/package.scala similarity index 100% rename from hudi-spark/src/main/scala/org/apache/hudi/package.scala rename to hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/package.scala diff --git a/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java similarity index 100% rename from hudi-spark/src/test/java/HoodieJavaApp.java rename to hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java diff --git a/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java similarity index 100% rename from hudi-spark/src/test/java/HoodieJavaGenerateApp.java rename to hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java diff --git a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java similarity index 99% rename from hudi-spark/src/test/java/HoodieJavaStreamingApp.java rename to hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 606490f444e26..1df12a35032ab 100644 --- a/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -43,7 +43,7 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.OutputMode; -import org.apache.spark.sql.streaming.ProcessingTime; +import org.apache.spark.sql.streaming.Trigger; import java.util.List; import java.util.concurrent.ExecutorService; @@ -366,7 +366,7 @@ public void stream(Dataset streamingInput, String operationType, String che .outputMode(OutputMode.Append()); updateHiveSyncConfig(writer); - StreamingQuery query = writer.trigger(new ProcessingTime(500)).start(tablePath); + StreamingQuery query = writer.trigger(Trigger.ProcessingTime(500)).start(tablePath); query.awaitTermination(streamingDurationInMs); } diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java similarity index 100% rename from hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java similarity index 100% rename from hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieDatasetBulkInsertHelper.java diff --git a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java similarity index 95% rename from hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java index 7e13a5e4b0fa4..dbf95de7986d3 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java @@ -28,6 +28,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.bootstrap.FileStatusUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; @@ -372,7 +373,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -390,7 +392,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -406,7 +409,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -423,7 +427,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -438,7 +443,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta reloadInputFormats(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -455,7 +461,8 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta seenKeys = new HashSet<>(); records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(metaClient.getFs(), basePath, false).stream() + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); diff --git a/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java similarity index 100% rename from hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java diff --git a/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java similarity index 100% rename from hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java rename to hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java diff --git a/hudi-spark/src/test/resources/exampleSchema.txt b/hudi-spark-datasource/hudi-spark/src/test/resources/exampleSchema.txt similarity index 100% rename from hudi-spark/src/test/resources/exampleSchema.txt rename to hudi-spark-datasource/hudi-spark/src/test/resources/exampleSchema.txt diff --git a/hudi-spark/src/test/resources/log4j-surefire-quiet.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties similarity index 100% rename from hudi-spark/src/test/resources/log4j-surefire-quiet.properties rename to hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire-quiet.properties diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000000000..32af462093ae5 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/log4j-surefire.properties @@ -0,0 +1,31 @@ +### +# 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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache=INFO +log4j.logger.org.apache.hudi=DEBUG +log4j.logger.org.apache.hadoop.hbase=ERROR + +# A1 is set to be a ConsoleAppender. +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +# A1 uses PatternLayout. +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter +log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true +log4j.appender.CONSOLE.filter.a.LevelMin=WARN +log4j.appender.CONSOLE.filter.a.LevelMax=FATAL diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala similarity index 100% rename from hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionHelper.scala diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala similarity index 86% rename from hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala index 99e1297f6ae47..4c69950c0505e 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceDefaults.scala @@ -17,12 +17,14 @@ package org.apache.hudi +import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.config.TypedProperties -import org.apache.hudi.common.model.{EmptyHoodieRecordPayload, HoodieKey, OverwriteWithLatestAvroPayload} +import org.apache.hudi.common.model.{BaseAvroPayload, DefaultHoodieRecordPayload, EmptyHoodieRecordPayload, HoodieKey, HoodiePayloadProps, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.testutils.SchemaTestUtil import org.apache.hudi.common.util.Option +import org.apache.hudi.config.HoodiePayloadConfig import org.apache.hudi.exception.{HoodieException, HoodieKeyException} import org.apache.hudi.keygen._ import org.apache.hudi.testutils.KeyGeneratorTestUtilities @@ -31,6 +33,8 @@ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{BeforeEach, Test} import org.scalatest.Assertions.fail +import scala.collection.JavaConverters.mapAsJavaMapConverter + /** * Tests on the default key generator, payload classes. */ @@ -567,6 +571,62 @@ class TestDataSourceDefaults { assertEquals("field2", combinedGR21.get("field1").toString) } + @Test def testOverwriteWithLatestAvroPayloadCombineAndGetUpdateValue() = { + val baseOrderingVal: Object = baseRecord.get("favoriteIntNumber") + val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema() + val props = new TypedProperties() + props.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "favoriteIntNumber"); + + val basePayload = new OverwriteWithLatestAvroPayload(baseRecord, HoodieAvroUtils.convertValueForSpecificDataTypes(fieldSchema, baseOrderingVal).asInstanceOf[Comparable[_]]) + + val laterRecord = SchemaTestUtil + .generateAvroRecordFromJson(schema, 2, "001", "f1") + val laterOrderingVal: Object = laterRecord.get("favoriteIntNumber") + val newerPayload = new OverwriteWithLatestAvroPayload(laterRecord, HoodieAvroUtils.convertValueForSpecificDataTypes(fieldSchema, laterOrderingVal).asInstanceOf[Comparable[_]]) + + // it will provide the record with greatest combine value + val preCombinedPayload = basePayload.preCombine(newerPayload) + val precombinedGR = preCombinedPayload.getInsertValue(schema).get().asInstanceOf[GenericRecord] + assertEquals("field2", precombinedGR.get("field1").toString) + } + + @Test def testDefaultHoodieRecordPayloadCombineAndGetUpdateValue() = { + val baseOrderingVal: Object = baseRecord.get("favoriteIntNumber") + val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema() + val props = new TypedProperties() + props.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "favoriteIntNumber"); + + val laterRecord = SchemaTestUtil + .generateAvroRecordFromJson(schema, 2, "001", "f1") + val laterOrderingVal: Object = laterRecord.get("favoriteIntNumber") + + val earlierRecord = SchemaTestUtil + .generateAvroRecordFromJson(schema, 1, "000", "f1") + val earlierOrderingVal: Object = earlierRecord.get("favoriteIntNumber") + + val laterPayload = new DefaultHoodieRecordPayload(laterRecord, + HoodieAvroUtils.convertValueForSpecificDataTypes(fieldSchema, laterOrderingVal).asInstanceOf[Comparable[_]]) + + val earlierPayload = new DefaultHoodieRecordPayload(earlierRecord, + HoodieAvroUtils.convertValueForSpecificDataTypes(fieldSchema, earlierOrderingVal).asInstanceOf[Comparable[_]]) + + // it will provide the record with greatest combine value + val preCombinedPayload = laterPayload.preCombine(earlierPayload) + val precombinedGR = preCombinedPayload.getInsertValue(schema).get().asInstanceOf[GenericRecord] + assertEquals("field2", precombinedGR.get("field1").toString) + assertEquals(laterOrderingVal, precombinedGR.get("favoriteIntNumber")) + + val earlierWithLater = earlierPayload.combineAndGetUpdateValue(laterRecord, schema, props) + val earlierwithLaterGR = earlierWithLater.get().asInstanceOf[GenericRecord] + assertEquals("field2", earlierwithLaterGR.get("field1").toString) + assertEquals(laterOrderingVal, earlierwithLaterGR.get("favoriteIntNumber")) + + val laterWithEarlier = laterPayload.combineAndGetUpdateValue(earlierRecord, schema, props) + val laterWithEarlierGR = laterWithEarlier.get().asInstanceOf[GenericRecord] + assertEquals("field2", laterWithEarlierGR.get("field1").toString) + assertEquals(laterOrderingVal, laterWithEarlierGR.get("favoriteIntNumber")) + } + @Test def testEmptyHoodieRecordPayload() = { val emptyPayload1 = new EmptyHoodieRecordPayload(baseRecord, 1) val laterRecord = SchemaTestUtil diff --git a/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala similarity index 100% rename from hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala similarity index 100% rename from hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala similarity index 58% rename from hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 5b746deb0d51a..b15a7d470a6cf 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -19,15 +19,21 @@ package org.apache.hudi.functional import java.sql.{Date, Timestamp} +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieInstant +import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.spark.sql._ -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.sql.types.{DataTypes, DateType, IntegerType, StringType, StructField, StructType, TimestampType} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.JavaConversions._ @@ -46,6 +52,9 @@ class TestCOWDataSource extends HoodieClientTestBase { HoodieWriteConfig.TABLE_NAME -> "hoodie_test" ) + val verificationCol: String = "driver" + val updatedVerificationVal: String = "driver_update" + @BeforeEach override def setUp() { initPath() initSparkContexts() @@ -73,13 +82,18 @@ class TestCOWDataSource extends HoodieClientTestBase { assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) } - @Test def testCopyOnWriteStorage() { + @ParameterizedTest + //TODO(metadata): Needs HUDI-1459 to be fixed + //@ValueSource(booleans = Array(true, false)) + @ValueSource(booleans = Array(false)) + def testCopyOnWriteStorage(isMetadataEnabled: Boolean) { // Insert Operation val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) inputDF1.write.format("org.apache.hudi") .options(commonOpts) .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Overwrite) .save(basePath) @@ -87,26 +101,44 @@ class TestCOWDataSource extends HoodieClientTestBase { val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath) // Snapshot query - val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*/*") + val snapshotDF1 = spark.read.format("org.apache.hudi") + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) + .load(basePath + "/*/*/*/*") assertEquals(100, snapshotDF1.count()) + // Upsert based on the written table with Hudi metadata columns + val verificationRowKey = snapshotDF1.limit(1).select("_row_key").first.getString(0) + val updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + + updateDf.write.format("org.apache.hudi") + .options(commonOpts) + .mode(SaveMode.Append) + .save(basePath) + val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) + + val snapshotDF2 = spark.read.format("hudi").load(basePath + "/*/*/*/*") + assertEquals(100, snapshotDF2.count()) + assertEquals(updatedVerificationVal, snapshotDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) + + // Upsert Operation without Hudi metadata columns val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList - val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2 , 2)) val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() - // Upsert Operation inputDF2.write.format("org.apache.hudi") .options(commonOpts) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Append) .save(basePath) - val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size()) + val commitInstantTime3 = HoodieDataSourceHelpers.latestCommit(fs, basePath) + assertEquals(3, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size()) // Snapshot Query - val snapshotDF2 = spark.read.format("org.apache.hudi") + val snapshotDF3 = spark.read.format("org.apache.hudi") + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .load(basePath + "/*/*/*/*") - assertEquals(100, snapshotDF2.count()) // still 100, since we only updated + assertEquals(100, snapshotDF3.count()) // still 100, since we only updated // Read Incremental Query // we have 2 commits, try pulling the first commit (which is not the latest) @@ -126,24 +158,25 @@ class TestCOWDataSource extends HoodieClientTestBase { val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1)) emptyDF.write.format("org.apache.hudi") .options(commonOpts) + .option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled) .mode(SaveMode.Append) .save(basePath) // pull the latest commit val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2) .load(basePath) assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect() assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + assertEquals(commitInstantTime3, countsPerCommit(0).get(0)) // pull the latest commit within certain partitions val hoodieIncViewDF3 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2) .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*") .load(basePath) assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count()) @@ -156,6 +189,80 @@ class TestCOWDataSource extends HoodieClientTestBase { assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled } + @Test def testOverWriteModeUseReplaceAction(): Unit = { + val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + val records2 = recordsToStrings(dataGen.generateInserts("002", 5)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true) + val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray + .map(instant => (instant.asInstanceOf[HoodieInstant]).getAction) + assertEquals(2, commits.size) + assertEquals("commit", commits(0)) + assertEquals("replacecommit", commits(1)) + } + + @Test def testOverWriteModeUseReplaceActionOnDisJointPartitions(): Unit = { + // step1: Write 5 records to hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH + val records1 = recordsToStrings(dataGen.generateInsertsForPartition("001", 5, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + + // step2: Write 7 more rectestOverWriteModeUseReplaceActionords using SaveMode.Overwrite for partition2 DEFAULT_SECOND_PARTITION_PATH + val records2 = recordsToStrings(dataGen.generateInsertsForPartition("002", 7, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + val allRecords = spark.read.format("org.apache.hudi").load(basePath + "/*/*/*") + allRecords.registerTempTable("tmpTable") + + spark.sql(String.format("select count(*) from tmpTable")).show() + + // step3: Query the rows count from hoodie table for partition1 DEFAULT_FIRST_PARTITION_PATH + val recordCountForParititon1 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).collect() + assertEquals("0", recordCountForParititon1(0).get(0).toString) + + // step4: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH + val recordCountForParititon2 = spark.sql(String.format("select count(*) from tmpTable where partition = '%s'", HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).collect() + assertEquals("7", recordCountForParititon2(0).get(0).toString) + + // step5: Query the rows count from hoodie table + val recordCount = spark.sql(String.format("select count(*) from tmpTable")).collect() + assertEquals("7", recordCountForParititon2(0).get(0).toString) + + // step6: Query the rows count from hoodie table for partition1 DEFAULT_SECOND_PARTITION_PATH using spark.collect and then filter mode + val recordsForPartitionColumn = spark.sql(String.format("select partition from tmpTable")).collect() + val filterSecondPartitionCount = recordsForPartitionColumn.filter(row => row.get(0).equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)).size + assertEquals(7,filterSecondPartitionCount) + + val metaClient = new HoodieTableMetaClient(spark.sparkContext.hadoopConfiguration, basePath, true) + val commits = metaClient.getActiveTimeline.filterCompletedInstants().getInstants.toArray + .map(instant => instant.asInstanceOf[HoodieInstant].getAction) + assertEquals(2, commits.size) + assertEquals("commit", commits(0)) + assertEquals("replacecommit", commits(1)) + } + @Test def testDropInsertDup(): Unit = { val insert1Cnt = 10 val insert2DupKeyCnt = 9 @@ -228,4 +335,17 @@ class TestCOWDataSource extends HoodieClientTestBase { } }) } + + @Test def testWithAutoCommitOn(): Unit = { + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, "true") + .mode(SaveMode.Overwrite) + .save(basePath) + + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) + } } diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala similarity index 59% rename from hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala index f24e5ad5bacc1..2a6a0a71360af 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala @@ -17,9 +17,6 @@ package org.apache.hudi.functional -import java.time.Instant -import java.util.Collections - import collection.JavaConverters._ import org.apache.hadoop.fs.FileSystem import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider @@ -37,10 +34,13 @@ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.io.TempDir +import java.time.Instant +import java.util.Collections + class TestDataSourceForBootstrap { var spark: SparkSession = _ - val commonOpts = Map( + val commonOpts: Map[String, String] = Map( HoodieWriteConfig.INSERT_PARALLELISM -> "4", HoodieWriteConfig.UPSERT_PARALLELISM -> "4", HoodieWriteConfig.DELETE_PARALLELISM -> "4", @@ -56,6 +56,14 @@ class TestDataSourceForBootstrap { var srcPath: String = _ var fs: FileSystem = _ + val partitionPaths: List[String] = List("2020-04-01", "2020-04-02", "2020-04-03") + val numRecords: Int = 100 + val numRecordsUpdate: Int = 10 + val verificationRowKey: String = "trip_0" + val verificationCol: String = "driver" + val originalVerificationVal: String = "driver_0" + val updatedVerificationVal: String = "driver_update" + @BeforeEach def initialize(@TempDir tempDir: java.nio.file.Path) { spark = SparkSession.builder .appName("Hoodie Datasource test") @@ -83,7 +91,6 @@ class TestDataSourceForBootstrap { @Test def testMetadataBootstrapCOWNonPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, Collections.emptyList(), jsc, @@ -96,20 +103,7 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.NonpartitionedKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") @@ -117,18 +111,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, Collections.emptyList(), jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator") .mode(SaveMode.Append) .save(basePath) @@ -141,36 +131,11 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF1.count()) assertEquals(numRecordsUpdate, hoodieROViewDF1.filter(s"timestamp == $updateTimestamp").count()) - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) - - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) - - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); - - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = false, isHiveStylePartitioned = false) } @Test def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -184,20 +149,7 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") @@ -205,18 +157,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") // Required because source data is hive style partitioned .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true") @@ -231,49 +179,14 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF2.count()) assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) - - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) - - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); - - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) - - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/datestr=2020-04-02/*") - .load(basePath) - - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = true) } @Test def testMetadataBootstrapCOWPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) - var sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, + val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, spark.sqlContext) // Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence @@ -288,92 +201,56 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") assertEquals(numRecords, hoodieROViewDF1.count()) - // Perform upsert - val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 - var updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, - jsc, spark.sqlContext) - - updateDF.write + // Perform upsert based on the written bootstrap table + val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + updateDf1.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option("hoodie.upsert.shuffle.parallelism", "4") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) - val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + // Read table after upsert and verify the updated value assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) - - // Read table after upsert and verify count val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF2.count()) - assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) - - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) + hoodieROViewDF2.collect() + assertEquals(updatedVerificationVal, hoodieROViewDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) + // Perform upsert based on the source data + val updateTimestamp = Instant.now.toEpochMilli + val updateDF2 = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, + jsc, spark.sqlContext) - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); + updateDF2.write + .format("hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") + .mode(SaveMode.Append) + .save(basePath) - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + val commitInstantTime3: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*") - .load(basePath) + // Read table after upsert and verify count + val hoodieROViewDF3 = spark.read.format("hudi").load(basePath + "/*") + assertEquals(numRecords, hoodieROViewDF3.count()) + assertEquals(numRecordsUpdate, hoodieROViewDF3.filter(s"timestamp == $updateTimestamp").count()) - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime3, isPartitioned = true, isHiveStylePartitioned = false) } @Test def testMetadataBootstrapMORPartitionedInlineCompactionOn(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -391,21 +268,7 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi") @@ -416,18 +279,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .option(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true") .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1") @@ -449,8 +308,6 @@ class TestDataSourceForBootstrap { @Test def testMetadataBootstrapMORPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -468,64 +325,64 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + + // Read bootstrapped table and verify count + val hoodieROViewDF1 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, + DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) + .load(basePath + "/*") + assertEquals(numRecords, hoodieROViewDF1.count()) + + // Perform upsert based on the written bootstrap table + val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + updateDf1.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") + .mode(SaveMode.Append) .save(basePath) - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) - - // Read bootstrapped table and verify count - val hoodieROViewDF1 = spark.read.format("hudi") + // Read table after upsert and verify the value + assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) + val hoodieROViewDF2 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF1.count()) + hoodieROViewDF2.collect() + assertEquals(originalVerificationVal, hoodieROViewDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) - // Perform upsert + // Perform upsert based on the source data val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 - val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, + val updateDF2 = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) - updateDF.write + updateDF2.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) - // Expect 1 new commit since meta bootstrap - delta commit (because inline compaction is off) - assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) + // Expect 2 new commit since meta bootstrap - 2 delta commits (because inline compaction is off) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) // Read table after upsert and verify count. Since we have inline compaction off the RO view will have // no updated rows. - val hoodieROViewDF2 = spark.read.format("hudi") + val hoodieROViewDF3 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF2.count()) - assertEquals(0, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) + assertEquals(numRecords, hoodieROViewDF3.count()) + assertEquals(0, hoodieROViewDF3.filter(s"timestamp == $updateTimestamp").count()) } @Test def testFullBootstrapCOWPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -547,11 +404,8 @@ class TestDataSourceForBootstrap { bootstrapDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName) .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, classOf[FullRecordBootstrapModeSelector].getName) @@ -568,18 +422,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) @@ -592,39 +442,64 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF2.count()) assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = false) + } + + def runMetadataBootstrapAndVerifyCommit(tableType: String): String = { + val bootstrapDF = spark.emptyDataFrame + bootstrapDF.write + .format("hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName) + .mode(SaveMode.Overwrite) + .save(basePath) + + val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + commitInstantTime1 + } + + def verifyIncrementalViewResult(bootstrapCommitInstantTime: String, latestCommitInstantTime: String, + isPartitioned: Boolean, isHiveStylePartitioned: Boolean): Unit = { // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written // during bootstrap val hoodieIncViewDF1 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) .load(basePath) assertEquals(numRecords, hoodieIncViewDF1.count()) var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) + assertEquals(bootstrapCommitInstantTime, countsPerCommit(0).get(0)) - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit + // incrementally pull only changes after bootstrap commit, which would pull only the updated records in the + // later commits val hoodieIncViewDF2 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) .load(basePath); assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) - - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*") - .load(basePath) - - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + assertEquals(latestCommitInstantTime, countsPerCommit(0).get(0)) + + if (isPartitioned) { + val relativePartitionPath = if (isHiveStylePartitioned) "/datestr=2020-04-02/*" else "/2020-04-02/*" + // pull the update commits within certain partitions + val hoodieIncViewDF3 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) + .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, relativePartitionPath) + .load(basePath) + + assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), + hoodieIncViewDF3.count()) + } } } diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala similarity index 71% rename from hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 5938ee54e110b..1ea6ceb879fb9 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -19,7 +19,7 @@ package org.apache.hudi.functional import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.testutils.HoodieTestDataGenerator -import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.testutils.HoodieClientTestBase @@ -29,6 +29,7 @@ import org.apache.spark.sql.functions._ import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} + import scala.collection.JavaConversions._ /** @@ -47,6 +48,9 @@ class TestMORDataSource extends HoodieClientTestBase { HoodieWriteConfig.TABLE_NAME -> "hoodie_test" ) + val verificationCol: String = "driver" + val updatedVerificationVal: String = "driver_update" + @BeforeEach override def setUp() { initPath() initSparkContexts() @@ -86,7 +90,7 @@ class TestMORDataSource extends HoodieClientTestBase { val insertCommitTimes = hudiRODF1.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList assertEquals(List(insertCommitTime), insertCommitTimes) - // Upsert operation + // Upsert operation without Hudi metadata columns val records2 = recordsToStrings(dataGen.generateUniqueUpdates("002", 100)).toList val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) inputDF2.write.format("org.apache.hudi") @@ -101,6 +105,19 @@ class TestMORDataSource extends HoodieClientTestBase { .load(basePath + "/*/*/*/*") val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList assertEquals(List(updateCommitTime), updateCommitTimes) + + // Upsert based on the written table with Hudi metadata columns + val verificationRowKey = hudiSnapshotDF2.limit(1).select("_row_key").first.getString(0) + val inputDF3 = hudiSnapshotDF2.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + + inputDF3.write.format("org.apache.hudi") + .options(commonOpts) + .mode(SaveMode.Append) + .save(basePath) + + val hudiSnapshotDF3 = spark.read.format("hudi").load(basePath + "/*/*/*/*") + assertEquals(100, hudiSnapshotDF3.count()) + assertEquals(updatedVerificationVal, hudiSnapshotDF3.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) } @Test def testCount() { @@ -141,6 +158,39 @@ class TestMORDataSource extends HoodieClientTestBase { assertTrue(commit2Time > commit1Time) assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count()) + // incremental view + // base file only + val hudiIncDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit1Time) + .load(basePath) + assertEquals(100, hudiIncDF1.count()) + assertEquals(1, hudiIncDF1.select("_hoodie_commit_time").distinct().count()) + assertEquals(commit1Time, hudiIncDF1.select("_hoodie_commit_time").head().get(0).toString) + hudiIncDF1.show(1) + // log file only + val hudiIncDF2 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit1Time) + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit2Time) + .load(basePath) + assertEquals(100, hudiIncDF2.count()) + assertEquals(1, hudiIncDF2.select("_hoodie_commit_time").distinct().count()) + assertEquals(commit2Time, hudiIncDF2.select("_hoodie_commit_time").head().get(0).toString) + hudiIncDF2.show(1) + + // base file + log file + val hudiIncDF3 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit2Time) + .load(basePath) + assertEquals(100, hudiIncDF3.count()) + // log file being load + assertEquals(1, hudiIncDF3.select("_hoodie_commit_time").distinct().count()) + assertEquals(commit2Time, hudiIncDF3.select("_hoodie_commit_time").head().get(0).toString) + // Unmerge val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) @@ -177,6 +227,22 @@ class TestMORDataSource extends HoodieClientTestBase { assertEquals(50, hudiSnapshotDF3.join(hudiSnapshotDF2, Seq("_hoodie_record_key", "_hoodie_commit_time"), "inner").count()) + // incremental query from commit2Time + val hudiIncDF4 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time) + .load(basePath) + assertEquals(50, hudiIncDF4.count()) + + // skip merge incremental view + // including commit 2 and commit 3 + val hudiIncDF4SkipMerge = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) + .load(basePath) + assertEquals(200, hudiIncDF4SkipMerge.count()) + // Fourth Operation: // Insert records to a new partition. Produced a new parquet file. // SNAPSHOT view should read the latest log files from the default partition and parquet from the new partition. @@ -197,21 +263,50 @@ class TestMORDataSource extends HoodieClientTestBase { assertEquals(100, hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count()) + // Incremental query, 50 from log file, 100 from base file of the new partition. + val hudiIncDF5 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time) + .load(basePath) + assertEquals(150, hudiIncDF5.count()) + // Fifth Operation: // Upsert records to the new partition. Produced a newer version of parquet file. // SNAPSHOT view should read the latest log files from the default partition // and the latest parquet from the new partition. - val records5 = recordsToStrings(newDataGen.generateUpdates("005", 100)).toList + val records5 = recordsToStrings(newDataGen.generateUniqueUpdates("005", 50)).toList val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2)) inputDF5.write.format("org.apache.hudi") .options(commonOpts) - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Append) .save(basePath) + val commit5Time = HoodieDataSourceHelpers.latestCommit(fs, basePath) val hudiSnapshotDF5 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") assertEquals(200, hudiSnapshotDF5.count()) + + // Sixth Operation: + // Insert 2 records and trigger compaction. + val records6 = recordsToStrings(newDataGen.generateInserts("006", 2)).toList + val inputDF6: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records6, 2)) + inputDF6.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "true") + .mode(SaveMode.Append) + .save(basePath) + val commit6Time = HoodieDataSourceHelpers.latestCommit(fs, basePath) + val hudiSnapshotDF6 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/2020/01/10/*") + assertEquals(102, hudiSnapshotDF6.count()) + val hudiIncDF6 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit5Time) + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit6Time) + .load(basePath) + // compaction updated 150 rows + inserted 2 new row + assertEquals(152, hudiIncDF6.count()) } @Test @@ -260,6 +355,13 @@ class TestMORDataSource extends HoodieClientTestBase { .load(basePath + "/*/*/*/*") assertEquals(100, hudiSnapshotDF2Unmerge.count()) + // incremental query, read 50 delete records from log file and get 0 count. + val hudiIncDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time) + .load(basePath) + assertEquals(0, hudiIncDF1.count()) + // Third Operation: // Upsert 50 delete records to delete the reset // Snopshot view should read 0 record @@ -292,6 +394,8 @@ class TestMORDataSource extends HoodieClientTestBase { val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") + val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString + assertEquals(100, hudiSnapshotDF1.count()) // select nested columns with order different from the actual schema assertEquals("amount,currency,tip_history,_hoodie_commit_seqno", @@ -313,34 +417,43 @@ class TestMORDataSource extends HoodieClientTestBase { val hudiSnapshotDF2 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .load(basePath + "/*/*/*/*") - - val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString + val hudiIncDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .load(basePath) + val hudiIncDF1Skipmerge = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") + .load(basePath) + val hudiIncDF2 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit1Time) + .load(basePath) // filter first commit and only read log records assertEquals(50, hudiSnapshotDF2.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history") .filter(col("_hoodie_commit_time") > commit1Time).count()) + assertEquals(50, hudiIncDF1.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history") + .filter(col("_hoodie_commit_time") > commit1Time).count()) + assertEquals(50, hudiIncDF2 + .select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").count()) + assertEquals(150, hudiIncDF1Skipmerge + .select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").count()) // select nested columns with order different from the actual schema - assertEquals("amount,currency,tip_history,_hoodie_commit_seqno", - hudiSnapshotDF2 - .select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno") - .orderBy(desc("_hoodie_commit_seqno")) - .columns.mkString(",")) - - // Correctly loading type - val sampleRow = hudiSnapshotDF2 - .select("begin_lat", "current_date", "fare.currency", "tip_history", "nation") - .orderBy(desc("_hoodie_commit_time")) - .head() - assertEquals(sampleRow.getDouble(0), sampleRow.get(0)) - assertEquals(sampleRow.getLong(1), sampleRow.get(1)) - assertEquals(sampleRow.getString(2), sampleRow.get(2)) - assertEquals(sampleRow.getSeq(3), sampleRow.get(3)) - assertEquals(sampleRow.getStruct(4), sampleRow.get(4)) + verifySchemaAndTypes(hudiSnapshotDF1) + verifySchemaAndTypes(hudiSnapshotDF2) + verifySchemaAndTypes(hudiIncDF1) + verifySchemaAndTypes(hudiIncDF2) + verifySchemaAndTypes(hudiIncDF1Skipmerge) // make sure show() work - hudiSnapshotDF1.show(1) - hudiSnapshotDF2.show(1) + verifyShow(hudiSnapshotDF1) + verifyShow(hudiSnapshotDF2) + verifyShow(hudiIncDF1) + verifyShow(hudiIncDF2) + verifyShow(hudiIncDF1Skipmerge) } @Test @@ -388,4 +501,25 @@ class TestMORDataSource extends HoodieClientTestBase { hudiSnapshotDF1.show(1) hudiSnapshotDF2.show(1) } + + def verifySchemaAndTypes(df: DataFrame): Unit = { + assertEquals("amount,currency,tip_history,_hoodie_commit_seqno", + df.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno") + .orderBy(desc("_hoodie_commit_seqno")) + .columns.mkString(",")) + val sampleRow = df + .select("begin_lat", "current_date", "fare.currency", "tip_history", "nation") + .orderBy(desc("_hoodie_commit_time")) + .head() + assertEquals(sampleRow.getDouble(0), sampleRow.get(0)) + assertEquals(sampleRow.getLong(1), sampleRow.get(1)) + assertEquals(sampleRow.getString(2), sampleRow.get(2)) + assertEquals(sampleRow.getSeq(3), sampleRow.get(3)) + assertEquals(sampleRow.getStruct(4), sampleRow.get(4)) + } + + def verifyShow(df: DataFrame): Unit = { + df.show(1) + df.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").show(1) + } } diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala similarity index 53% rename from hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 226cf5313f5d0..fd768c669d198 100644 --- a/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -18,16 +18,20 @@ package org.apache.hudi.functional import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.common.model.FileSlice import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestTable} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings -import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.config.{HoodieClusteringConfig, HoodieStorageConfig, HoodieWriteConfig} import org.apache.hudi.exception.TableNotFoundException import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.log4j.LogManager import org.apache.spark.sql._ -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} +import org.apache.spark.sql.types.StructType +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} +import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import scala.collection.JavaConversions._ @@ -64,42 +68,49 @@ class TestStructuredStreaming extends HoodieClientTestBase { cleanupFileSystem() } - @Test - def testStructuredStreaming(): Unit = { - fs.delete(new Path(basePath), true) - val sourcePath = basePath + "/source" - val destPath = basePath + "/dest" - fs.mkdirs(new Path(sourcePath)) - - // First chunk of data - val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList - val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) - - // Second chunk of data - val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList - val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) - val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() - + def initStreamingWriteFuture(schema: StructType, sourcePath: String, destPath: String, hudiOptions: Map[String, String]): Future[Unit] = { // define the source of streaming val streamingInput = spark.readStream - .schema(inputDF1.schema) + .schema(schema) .json(sourcePath) - - val f1 = Future { + Future { println("streaming starting") //'writeStream' can be called only on streaming Dataset/DataFrame streamingInput .writeStream .format("org.apache.hudi") - .options(commonOpts) - .trigger(new ProcessingTime(100)) + .options(hudiOptions) + .trigger(Trigger.ProcessingTime(100)) .option("checkpointLocation", basePath + "/checkpoint") .outputMode(OutputMode.Append) .start(destPath) .awaitTermination(10000) println("streaming ends") } + } + + def initStreamingSourceAndDestPath(sourceDirName: String, destDirName: String): (String, String) = { + fs.delete(new Path(basePath), true) + val sourcePath = basePath + "/" + sourceDirName + val destPath = basePath + "/" + destDirName + fs.mkdirs(new Path(sourcePath)) + (sourcePath, destPath) + } + + @Test + def testStructuredStreaming(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + // First chunk of data + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + // Second chunk of data + val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() + + val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, commonOpts) val f2 = Future { inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) @@ -113,7 +124,7 @@ class TestStructuredStreaming extends HoodieClientTestBase { assert(hoodieROViewDF1.count() == 100) inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) - // wait for spark streaming to process one microbatch + // wait for spark streaming to process second microbatch waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5) val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, destPath) assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size()) @@ -177,4 +188,113 @@ class TestStructuredStreaming extends HoodieClientTestBase { if (!success) throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath) numInstants } + + def getInlineClusteringOpts( isInlineClustering: String, clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = { + commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING_PROP -> isInlineClustering, + HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP -> clusteringNumCommit, + HoodieStorageConfig.PARQUET_FILE_MAX_BYTES -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString + ) + } + + @Test + def testStructuredStreamingWithInlineClustering(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + + def checkClusteringResult(destPath: String):Unit = { + // check have schedule clustering and clustering file group to one + waitTillHasCompletedReplaceInstant(destPath, 120, 5) + metaClient.reloadActiveTimeline() + assertEquals(1, getLatestFileGroupsFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH).size) + } + structuredStreamingForTestClusteringRunner(sourcePath, destPath, true, + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult) + } + + @Test + def testStructuredStreamingWithoutInlineClustering(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + + def checkClusteringResult(destPath: String):Unit = { + val msg = "Should have replace commit completed" + assertThrows(classOf[IllegalStateException], new Executable { + override def execute(): Unit = { + waitTillHasCompletedReplaceInstant(destPath, 120, 5) + } + }, msg) + println(msg) + } + structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, checkClusteringResult) + } + + def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String, + isInlineClustering: Boolean, partitionOfRecords: String, checkClusteringResult: String => Unit): Unit = { + // First insert of data + val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, partitionOfRecords)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + // Second insert of data + val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, partitionOfRecords)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + + val hudiOptions = getInlineClusteringOpts(isInlineClustering.toString, "2", 100) + val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, hudiOptions) + + val f2 = Future { + inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process one microbatch + val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000")) + + inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process second microbatch + waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size()) + + // check have more than one file group + this.metaClient = new HoodieTableMetaClient(fs.getConf, destPath, true) + assertTrue(getLatestFileGroupsFileId(partitionOfRecords).size > 1) + + // check clustering result + checkClusteringResult(destPath) + + // check data correct after clustering + val hoodieROViewDF2 = spark.read.format("org.apache.hudi") + .load(destPath + "/*/*/*/*") + assertEquals(200, hoodieROViewDF2.count()) + } + Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf) + } + + private def getLatestFileGroupsFileId(partition: String):Array[String] = { + getHoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline, + HoodieTestTable.of(metaClient).listAllBaseFiles()) + tableView.getLatestFileSlices(partition) + .toArray().map(slice => slice.asInstanceOf[FileSlice].getFileGroupId.getFileId) + } + + @throws[InterruptedException] + private def waitTillHasCompletedReplaceInstant(tablePath: String, + timeoutSecs: Int, sleepSecsAfterEachRun: Int) = { + val beginTime = System.currentTimeMillis + var currTime = beginTime + val timeoutMsecs = timeoutSecs * 1000 + var success = false + while ({!success && (currTime - beginTime) < timeoutMsecs}) try { + this.metaClient.reloadActiveTimeline() + val completeReplaceSize = this.metaClient.getActiveTimeline.getCompletedReplaceTimeline().getInstants.toArray.size + println("completeReplaceSize:" + completeReplaceSize) + if (completeReplaceSize > 0) { + success = true + } + } catch { + case te: TableNotFoundException => + log.info("Got table not found exception. Retrying") + } finally { + Thread.sleep(sleepSecsAfterEachRun * 1000) + currTime = System.currentTimeMillis + } + if (!success) throw new IllegalStateException("Timed-out waiting for completing replace instant appear in " + tablePath) + } + } diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml new file mode 100644 index 0000000000000..1f6f7927a5a88 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -0,0 +1,240 @@ + + + + + hudi-spark-datasource + org.apache.hudi + 0.8.0-SNAPSHOT + + 4.0.0 + + hudi-spark2_${scala.binary.version} + 0.8.0-SNAPSHOT + + hudi-spark2_${scala.binary.version} + jar + + + ${project.parent.parent.basedir} + + + + + + src/main/resources + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + -nobootcp + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/lib + true + true + true + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${skip.hudi-spark2.unit.tests} + + + + org.apache.rat + apache-rat-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + org.jacoco + jacoco-maven-plugin + + + + + + + + org.scala-lang + scala-library + ${scala11.version} + + + + + org.apache.hudi + hudi-client-common + ${project.version} + + + org.apache.hudi + hudi-spark-client + ${project.version} + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-spark-common + ${project.version} + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark2.version} + true + + + + io.netty + netty + 3.9.9.Final + true + + + io.netty + netty-all + 4.1.17.Final + true + + + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-client + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-common + ${project.version} + tests + test-jar + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java similarity index 72% rename from hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java rename to hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java index 5fb71df777df4..526f0ce4783d6 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -21,11 +21,7 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.sources.DataSourceRegister; import org.apache.spark.sql.sources.v2.DataSourceOptions; import org.apache.spark.sql.sources.v2.DataSourceV2; @@ -40,14 +36,8 @@ /** * DataSource V2 implementation for managing internal write logic. Only called internally. */ -public class DefaultSource implements DataSourceV2, ReadSupport, WriteSupport, - DataSourceRegister { - - private static final Logger LOG = LogManager - .getLogger(DefaultSource.class); - - private SparkSession sparkSession = null; - private Configuration configuration = null; +public class DefaultSource extends BaseDefaultSource implements DataSourceV2, + ReadSupport, WriteSupport, DataSourceRegister { @Override public String shortName() { @@ -67,25 +57,11 @@ public DataSourceReader createReader(DataSourceOptions options) { @Override public Optional createWriter(String writeUUID, StructType schema, SaveMode mode, DataSourceOptions options) { - String instantTime = options.get(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY).get(); + String instantTime = options.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY).get(); String path = options.get("path").get(); String tblName = options.get(HoodieWriteConfig.TABLE_NAME).get(); HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, options.asMap()); return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(), getConfiguration())); } - - private SparkSession getSparkSession() { - if (sparkSession == null) { - sparkSession = SparkSession.builder().getOrCreate(); - } - return sparkSession; - } - - private Configuration getConfiguration() { - if (configuration == null) { - this.configuration = getSparkSession().sparkContext().hadoopConfiguration(); - } - return configuration; - } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..3ce8d776a8e68 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java @@ -0,0 +1,59 @@ +/* + * 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.internal; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.writer.DataWriter; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; + +/** + * Hoodie's Implementation of {@link DataWriter}. This is used in data source implementation for bulk insert. + */ +public class HoodieBulkInsertDataInternalWriter implements DataWriter { + + private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper; + + public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, + StructType structType) { + this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable, + writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType); + } + + @Override + public void write(InternalRow record) throws IOException { + bulkInsertWriterHelper.write(record); + } + + @Override + public WriterCommitMessage commit() throws IOException { + return new HoodieWriterCommitMessage(bulkInsertWriterHelper.getWriteStatuses()); + } + + @Override + public void abort() { + bulkInsertWriterHelper.abort(); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java similarity index 100% rename from hudi-spark/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java rename to hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriterFactory.java diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java new file mode 100644 index 0000000000000..4b3dafc6264f7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -0,0 +1,90 @@ +/* + * 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.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; +import org.apache.spark.sql.sources.v2.writer.DataWriterFactory; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation + * of bulk insert. + */ +public class HoodieDataSourceInternalWriter implements DataSourceWriter { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper; + + public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession sparkSession, Configuration configuration) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, + sparkSession, configuration); + } + + @Override + public DataWriterFactory createWriterFactory() { + dataSourceInternalWriterHelper.createInflightCommit(); + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { + return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), + writeConfig, instantTime, structType); + } else { + throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported "); + } + } + + @Override + public boolean useCommitCoordinator() { + return dataSourceInternalWriterHelper.useCommitCoordinator(); + } + + @Override + public void onDataWriterCommit(WriterCommitMessage message) { + dataSourceInternalWriterHelper.onDataWriterCommit(message.toString()); + } + + @Override + public void commit(WriterCommitMessage[] messages) { + List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); + } + + @Override + public void abort(WriterCommitMessage[] messages) { + dataSourceInternalWriterHelper.abort(); + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java similarity index 72% rename from hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java rename to hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java index 757000c57c1dd..240e4b9819413 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java @@ -18,28 +18,18 @@ package org.apache.hudi.internal; -import java.util.ArrayList; -import java.util.List; import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import java.util.List; + /** * Hoodie's {@link WriterCommitMessage} used in datasource implementation. */ -public class HoodieWriterCommitMessage implements WriterCommitMessage { - - private List writeStatuses = new ArrayList<>(); +public class HoodieWriterCommitMessage extends BaseWriterCommitMessage + implements WriterCommitMessage { public HoodieWriterCommitMessage(List writeStatuses) { - this.writeStatuses = writeStatuses; - } - - public List getWriteStatuses() { - return writeStatuses; - } - - @Override - public String toString() { - return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}'; + super(writeStatuses); } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2RowDeserializer.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2RowDeserializer.scala new file mode 100644 index 0000000000000..84fe4c3e8b28b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/hudi/Spark2RowDeserializer.scala @@ -0,0 +1,30 @@ +/* + * 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 + +import org.apache.hudi.client.utils.SparkRowDeserializer + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +class Spark2RowDeserializer(val encoder: ExpressionEncoder[Row]) extends SparkRowDeserializer { + def deserializeRow(internalRow: InternalRow): Row = { + encoder.fromRow(internalRow) + } +} diff --git a/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..0b021abeb3b5f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -0,0 +1,149 @@ +/* + * 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.internal; + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; +import static org.junit.jupiter.api.Assertions.fail; + +/** + * Unit tests {@link HoodieBulkInsertDataInternalWriter}. + */ +public class TestHoodieBulkInsertDataInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { + + @Test + public void testDataInternalWriter() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + // execute N rounds + for (int i = 0; i < 5; i++) { + String instantTime = "00" + i; + // init writer + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + + int size = 10 + RANDOM.nextInt(1000); + // write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file + int batches = 5; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + BaseWriterCommitMessage commitMetadata = (BaseWriterCommitMessage) writer.commit(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); + + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames); + + // verify rows + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); + assertOutput(totalInputRows, result, instantTime, fileNames); + } + } + + + /** + * Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected + * to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk. + */ + @Test + public void testGlobalFailure() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; + + String instantTime = "001"; + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + + int size = 10 + RANDOM.nextInt(100); + int totalFailures = 5; + // Generate first batch of valid rows + Dataset inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false); + List internalRows = toInternalRows(inputRows, ENCODER); + + // generate some failures rows + for (int i = 0; i < totalFailures; i++) { + internalRows.add(getInternalRowWithError(partitionPath)); + } + + // generate 2nd batch of valid rows + Dataset inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false); + internalRows.addAll(toInternalRows(inputRows2, ENCODER)); + + // issue writes + try { + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + fail("Should have failed"); + } catch (Throwable e) { + // expected + } + + BaseWriterCommitMessage commitMetadata = (BaseWriterCommitMessage) writer.commit(); + + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames); + + // verify rows + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); + assertOutput(inputRows, result, instantTime, fileNames); + } + + private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWriter writer) + throws Exception { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java similarity index 71% rename from hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java rename to hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java index 89d748f671aac..184ff771cef4a 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -18,61 +18,35 @@ package org.apache.hudi.internal; -import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; /** * Unit tests {@link HoodieDataSourceInternalWriter}. */ -public class TestHoodieDataSourceInternalWriter extends HoodieClientTestHarness { - - private static final Random RANDOM = new Random(); - - @BeforeEach - public void setUp() throws Exception { - initSparkContexts("TestHoodieDataSourceInternalWriter"); - initPath(); - initFileSystem(); - initTestDataGenerator(); - initMetaClient(); - } - - @AfterEach - public void tearDown() throws Exception { - cleanupResources(); - } +public class TestHoodieDataSourceInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { @Test - public void testDataSourceWriter() throws IOException { + public void testDataSourceWriter() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); String instantTime = "001"; @@ -81,7 +55,7 @@ public void testDataSourceWriter() throws IOException { new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); - List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS; List partitionPathsAbs = new ArrayList<>(); for (String partitionPath : partitionPaths) { partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); @@ -109,12 +83,12 @@ public void testDataSourceWriter() throws IOException { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } @Test - public void testMultipleDataSourceWrites() throws IOException { + public void testMultipleDataSourceWrites() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); int partitionCounter = 0; @@ -152,13 +126,13 @@ public void testMultipleDataSourceWrites() throws IOException { Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } } @Test - public void testLargeWrites() throws IOException { + public void testLargeWrites() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); int partitionCounter = 0; @@ -196,8 +170,8 @@ public void testLargeWrites() throws IOException { Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } } @@ -208,7 +182,7 @@ public void testLargeWrites() throws IOException { * verify only records from batch1 is available to read */ @Test - public void testAbort() throws IOException { + public void testAbort() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); @@ -247,8 +221,8 @@ public void testAbort() throws IOException { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify rows - assertOutput(totalInputRows, result, instantTime0); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); // 2nd batch. abort in the end String instantTime1 = "00" + 1; @@ -271,51 +245,14 @@ public void testAbort() throws IOException { result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify rows // only rows from first batch should be present - assertOutput(totalInputRows, result, instantTime0); + assertOutput(totalInputRows, result, instantTime0, Option.empty()); } - private void writeRows(Dataset inputRows, DataWriter writer) throws IOException { + private void writeRows(Dataset inputRows, DataWriter writer) throws Exception { List internalRows = toInternalRows(inputRows, ENCODER); // issue writes for (InternalRow internalRow : internalRows) { writer.write(internalRow); } } - - private void assertWriteStatuses(List writeStatuses, int batches, int size) { - assertEquals(batches, writeStatuses.size()); - int counter = 0; - for (HoodieInternalWriteStatus writeStatus : writeStatuses) { - assertEquals(writeStatus.getPartitionPath(), HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]); - assertEquals(writeStatus.getTotalRecords(), size); - assertEquals(writeStatus.getFailedRowsSize(), 0); - assertEquals(writeStatus.getTotalErrorRecords(), 0); - assertFalse(writeStatus.hasErrors()); - assertNull(writeStatus.getGlobalError()); - assertNotNull(writeStatus.getFileId()); - String fileId = writeStatus.getFileId(); - HoodieWriteStat writeStat = writeStatus.getStat(); - assertEquals(size, writeStat.getNumInserts()); - assertEquals(size, writeStat.getNumWrites()); - assertEquals(fileId, writeStat.getFileId()); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); - assertEquals(0, writeStat.getNumDeletes()); - assertEquals(0, writeStat.getNumUpdateWrites()); - assertEquals(0, writeStat.getTotalWriteErrors()); - } - } - - private void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime) { - // verify 3 meta fields that are filled in within create handle - actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); - }); - - // after trimming 2 of the meta fields, rest of the fields should match - Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - assertEquals(0, trimmedActual.except(trimmedExpected).count()); - } } diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml new file mode 100644 index 0000000000000..9ea7fb2838fdf --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -0,0 +1,224 @@ + + + + + hudi-spark-datasource + org.apache.hudi + 0.8.0-SNAPSHOT + + 4.0.0 + + hudi-spark3_2.12 + 0.8.0-SNAPSHOT + + hudi-spark3_2.12 + jar + + + ${project.parent.parent.basedir} + + + + + + src/main/resources + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + -nobootcp + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/lib + true + true + true + + + + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + org.jacoco + jacoco-maven-plugin + + + + + + + org.scala-lang + scala-library + ${scala12.version} + + + + org.apache.spark + spark-sql_2.12 + ${spark3.version} + true + + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.spark3.version} + + + com.fasterxml.jackson.core + jackson-annotations + ${fasterxml.spark3.version} + + + com.fasterxml.jackson.core + jackson-core + ${fasterxml.spark3.version} + + + + org.apache.hudi + hudi-spark-client + ${project.version} + + + org.apache.hudi + hudi-spark-common + ${project.version} + + + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-client + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-common + ${project.version} + tests + test-jar + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java new file mode 100644 index 0000000000000..d59b5ad5c2935 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.BaseDefaultSource; +import org.apache.hudi.internal.DataSourceInternalWriterHelper; + +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +import java.util.Map; + +/** + * DataSource V2 implementation for managing internal write logic. Only called internally. + * This class is only compatible with datasource V2 API in Spark 3. + */ +public class DefaultSource extends BaseDefaultSource implements TableProvider { + + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + return StructType.fromDDL(options.get(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL)); + } + + @Override + public Table getTable(StructType schema, Transform[] partitioning, Map properties) { + String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY); + String path = properties.get("path"); + String tblName = properties.get(HoodieWriteConfig.TABLE_NAME); + HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, properties); + return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(), + getConfiguration()); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..f67187c2718a8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.BulkInsertDataInternalWriterHelper; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; + +/** + * Hoodie's Implementation of {@link DataWriter}. This is used in data source "hudi.spark3.internal" implementation for bulk insert. + */ +public class HoodieBulkInsertDataInternalWriter implements DataWriter { + + private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper; + + public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, StructType structType) { + this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable, + writeConfig, instantTime, taskPartitionId, taskId, 0, structType); + } + + @Override + public void write(InternalRow record) throws IOException { + bulkInsertWriterHelper.write(record); + } + + @Override + public WriterCommitMessage commit() throws IOException { + return new HoodieWriterCommitMessage(bulkInsertWriterHelper.getWriteStatuses()); + } + + @Override + public void abort() { + bulkInsertWriterHelper.abort(); + } + + @Override + public void close() throws IOException { + bulkInsertWriterHelper.close(); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java new file mode 100644 index 0000000000000..31b43ea7d45d6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.types.StructType; + +/** + * Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}. + */ +public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory { + + private final String instantTime; + private final HoodieTable hoodieTable; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + + public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, StructType structType) { + this.hoodieTable = hoodieTable; + this.writeConfig = writeConfig; + this.instantTime = instantTime; + this.structType = structType; + } + + @Override + public DataWriter createWriter(int partitionId, long taskId) { + return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, + structType); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java new file mode 100644 index 0000000000000..b0945156d703d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java @@ -0,0 +1,91 @@ +/* + * 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.spark3.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.DataSourceInternalWriterHelper; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; +import org.apache.spark.sql.types.StructType; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Implementation of {@link BatchWrite} for datasource "hudi.spark3.internal" to be used in datasource implementation + * of bulk insert. + */ +public class HoodieDataSourceInternalBatchWrite implements BatchWrite { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper; + + public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, + jss, hadoopConfiguration); + } + + @Override + public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { + dataSourceInternalWriterHelper.createInflightCommit(); + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { + return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), + writeConfig, instantTime, structType); + } else { + throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported "); + } + } + + @Override + public boolean useCommitCoordinator() { + return dataSourceInternalWriterHelper.useCommitCoordinator(); + } + + @Override + public void onDataWriterCommit(WriterCommitMessage message) { + dataSourceInternalWriterHelper.onDataWriterCommit(message.toString()); + } + + @Override + public void commit(WriterCommitMessage[] messages) { + List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); + } + + @Override + public void abort(WriterCommitMessage[] messages) { + dataSourceInternalWriterHelper.abort(); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java new file mode 100644 index 0000000000000..10e2e64f11387 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +/** + * Implementation of {@link WriteBuilder} for datasource "hudi.spark3.internal" to be used in datasource implementation + * of bulk insert. + */ +public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final SparkSession jss; + private final Configuration hadoopConfiguration; + + public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.jss = jss; + this.hadoopConfiguration = hadoopConfiguration; + } + + @Override + public BatchWrite buildForBatch() { + return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss, + hadoopConfiguration); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java new file mode 100644 index 0000000000000..f1fded033dcfc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.SupportsWrite; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +import java.util.HashSet; +import java.util.Set; + +/** + * Hoodie's Implementation of {@link SupportsWrite}. This is used in data source "hudi.spark3.internal" implementation for bulk insert. + */ +class HoodieDataSourceInternalTable implements SupportsWrite { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final SparkSession jss; + private final Configuration hadoopConfiguration; + + public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config, + StructType schema, SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = config; + this.structType = schema; + this.jss = jss; + this.hadoopConfiguration = hadoopConfiguration; + } + + @Override + public String name() { + return this.getClass().toString(); + } + + @Override + public StructType schema() { + return structType; + } + + @Override + public Set capabilities() { + return new HashSet() {{ + add(TableCapability.BATCH_WRITE); + add(TableCapability.TRUNCATE); + }}; + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) { + return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss, + hadoopConfiguration); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java new file mode 100644 index 0000000000000..7fe787deb8a08 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.internal.BaseWriterCommitMessage; +import org.apache.spark.sql.connector.write.WriterCommitMessage; + +import java.util.List; + +/** + * Hoodie's {@link WriterCommitMessage} used in datasource "hudi.spark3.internal" implementation. + */ +public class HoodieWriterCommitMessage extends BaseWriterCommitMessage + implements WriterCommitMessage { + + public HoodieWriterCommitMessage(List writeStatuses) { + super(writeStatuses); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3RowDeserializer.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3RowDeserializer.scala new file mode 100644 index 0000000000000..a0606553ff275 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/hudi/Spark3RowDeserializer.scala @@ -0,0 +1,33 @@ +/* + * 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 + +import org.apache.hudi.client.utils.SparkRowDeserializer + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +class Spark3RowDeserializer(val encoder: ExpressionEncoder[Row]) extends SparkRowDeserializer { + + private val deserializer: ExpressionEncoder.Deserializer[Row] = encoder.createDeserializer() + + def deserializeRow(internalRow: InternalRow): Row = { + deserializer.apply(internalRow) + } +} diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java similarity index 56% rename from hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java rename to hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java index 5a5d8b2700d4e..ffb649bd3970a 100644 --- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -16,28 +16,22 @@ * limitations under the License. */ -package org.apache.hudi.internal; +package org.apache.hudi.spark3.internal; -import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.HoodieBulkInsertInternalWriterTestBase; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Random; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; @@ -45,36 +39,16 @@ import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; /** * Unit tests {@link HoodieBulkInsertDataInternalWriter}. */ -public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarness { - - private static final Random RANDOM = new Random(); - - @BeforeEach - public void setUp() throws Exception { - initSparkContexts("TestHoodieBulkInsertDataInternalWriter"); - initPath(); - initFileSystem(); - initTestDataGenerator(); - initMetaClient(); - } - - @AfterEach - public void tearDown() throws Exception { - cleanupResources(); - } +public class TestHoodieBulkInsertDataInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { @Test - public void testDataInternalWriter() throws IOException { + public void testDataInternalWriter() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); @@ -82,7 +56,7 @@ public void testDataInternalWriter() throws IOException { for (int i = 0; i < 5; i++) { String instantTime = "00" + i; // init writer - HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE); int size = 10 + RANDOM.nextInt(1000); // write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file @@ -101,14 +75,14 @@ public void testDataInternalWriter() throws IOException { } HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); - List fileAbsPaths = new ArrayList<>(); - List fileNames = new ArrayList<>(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); // verify write statuses assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames); // verify rows - Dataset result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); assertOutput(totalInputRows, result, instantTime, fileNames); } } @@ -119,14 +93,14 @@ public void testDataInternalWriter() throws IOException { * to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk. */ @Test - public void testGlobalFailure() throws IOException { + public void testGlobalFailure() throws Exception { // init config and table HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; String instantTime = "001"; - HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE); + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE); int size = 10 + RANDOM.nextInt(100); int totalFailures = 5; @@ -155,60 +129,22 @@ public void testGlobalFailure() throws IOException { HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); - List fileAbsPaths = new ArrayList<>(); - List fileNames = new ArrayList<>(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); // verify write statuses assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames); // verify rows - Dataset result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); assertOutput(inputRows, result, instantTime, fileNames); } - private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWriter writer) throws IOException { + private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWriter writer) + throws Exception { List internalRows = toInternalRows(inputRows, ENCODER); // issue writes for (InternalRow internalRow : internalRows) { writer.write(internalRow); } } - - private void assertWriteStatuses(List writeStatuses, int batches, int size, List fileAbsPaths, List fileNames) { - assertEquals(batches, writeStatuses.size()); - int counter = 0; - for (HoodieInternalWriteStatus writeStatus : writeStatuses) { - // verify write status - assertEquals(writeStatus.getTotalRecords(), size); - assertNull(writeStatus.getGlobalError()); - assertEquals(writeStatus.getFailedRowsSize(), 0); - assertNotNull(writeStatus.getFileId()); - String fileId = writeStatus.getFileId(); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath()); - fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath()); - fileNames.add(writeStatus.getStat().getPath().substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1)); - HoodieWriteStat writeStat = writeStatus.getStat(); - assertEquals(size, writeStat.getNumInserts()); - assertEquals(size, writeStat.getNumWrites()); - assertEquals(fileId, writeStat.getFileId()); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); - assertEquals(0, writeStat.getNumDeletes()); - assertEquals(0, writeStat.getNumUpdateWrites()); - assertEquals(0, writeStat.getTotalWriteErrors()); - } - } - - private void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime, List fileNames) { - // verify 3 meta fields that are filled in within create handle - actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); - assertTrue(fileNames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)))); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); - }); - - // after trimming 2 of the meta fields, rest of the fields should match - Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - assertEquals(0, trimmedActual.except(trimmedExpected).count()); - } } diff --git a/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java new file mode 100644 index 0000000000000..69829ec281a49 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java @@ -0,0 +1,265 @@ +/* + * 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.spark3.internal; + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.HoodieBulkInsertInternalWriterTestBase; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestUtils; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; + +/** + * Unit tests {@link HoodieDataSourceInternalBatchWrite}. + */ +public class TestHoodieDataSourceInternalBatchWrite extends + HoodieBulkInsertInternalWriterTestBase { + + @Test + public void testDataSourceWriter() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String instantTime = "001"; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong()); + + String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS; + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(1000); + int batches = 5; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + List commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + + @Test + public void testMultipleDataSourceWrites() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + int partitionCounter = 0; + + // execute N rounds + for (int i = 0; i < 5; i++) { + String instantTime = "00" + i; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong()); + + int size = 10 + RANDOM.nextInt(1000); + int batches = 5; // one batch per partition + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); + + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + } + + @Test + public void testLargeWrites() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + int partitionCounter = 0; + + // execute N rounds + for (int i = 0; i < 3; i++) { + String instantTime = "00" + i; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong()); + + int size = 10000 + RANDOM.nextInt(10000); + int batches = 3; // one batch per partition + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); + + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + } + + /** + * Tests that DataSourceWriter.abort() will abort the written records of interest write and commit batch1 write and abort batch2 Read of entire dataset should show only records from batch1. + * commit batch1 + * abort batch2 + * verify only records from batch1 is available to read + */ + @Test + public void testAbort() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String instantTime0 = "00" + 0; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong()); + + List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(100); + int batches = 1; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + List commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + // commit 1st batch + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify rows + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + + // 2nd batch. abort in the end + String instantTime1 = "00" + 1; + dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong()); + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + } + + commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + // commit 1st batch + dataSourceInternalBatchWrite.abort(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify rows + // only rows from first batch should be present + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + } + + private void writeRows(Dataset inputRows, DataWriter writer) throws Exception { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } +} diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml new file mode 100644 index 0000000000000..aa1d82fde2f68 --- /dev/null +++ b/hudi-spark-datasource/pom.xml @@ -0,0 +1,39 @@ + + + + + hudi + org.apache.hudi + 0.8.0-SNAPSHOT + + 4.0.0 + + hudi-spark-datasource + pom + + + ${project.parent.basedir} + + + + hudi-spark-common + hudi-spark + hudi-spark2 + hudi-spark3 + + diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala deleted file mode 100644 index 26babd834b23d..0000000000000 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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 - -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hudi.common.model.HoodieRecord -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex} -import org.apache.spark.sql.types.{StringType, StructField, StructType} -import scala.collection.JavaConverters._ - - -object HoodieSparkUtils { - - def getMetaSchema: StructType = { - StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => { - StructField(col, StringType, nullable = true) - })) - } - - def checkAndGlobPathIfNecessary(paths: Seq[String], fs: FileSystem): Seq[Path] = { - paths.flatMap(path => { - val qualified = new Path(path).makeQualified(fs.getUri, fs.getWorkingDirectory) - val globPaths = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) - globPaths - }) - } - - def createInMemoryFileIndex(sparkSession: SparkSession, globbedPaths: Seq[Path]): InMemoryFileIndex = { - val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) - new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache) - } -} diff --git a/hudi-sync/hudi-dla-sync/pom.xml b/hudi-sync/hudi-dla-sync/pom.xml index 8a56b9d49e771..f269b237f0d32 100644 --- a/hudi-sync/hudi-dla-sync/pom.xml +++ b/hudi-sync/hudi-dla-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java index b04cc0a3241cd..5b50ada22c6c3 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java @@ -19,6 +19,8 @@ package org.apache.hudi.dla; import com.beust.jcommander.Parameter; + +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import java.io.Serializable; @@ -62,9 +64,18 @@ public class DLASyncConfig implements Serializable { @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") public Boolean skipROSuffix = false; + @Parameter(names = {"--skip-rt-sync"}, description = "Skip the RT table syncing") + public Boolean skipRTSync = false; + @Parameter(names = {"--hive-style-partitioning"}, description = "Use DLA hive style partitioning, true if like the following style: field1=value1/field2=value2") public Boolean useDLASyncHiveStylePartitioning = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -83,7 +94,10 @@ public static DLASyncConfig copy(DLASyncConfig cfg) { newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; newConfig.skipROSuffix = cfg.skipROSuffix; + newConfig.skipRTSync = cfg.skipRTSync; newConfig.useDLASyncHiveStylePartitioning = cfg.useDLASyncHiveStylePartitioning; + newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; + newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; return newConfig; } @@ -95,6 +109,8 @@ public String toString() { + ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning + ", useDLASyncHiveStylePartitioning=" + useDLASyncHiveStylePartitioning + + ", useFileListingFromMetadata=" + useFileListingFromMetadata + + ", verifyMetadataFileListing=" + verifyMetadataFileListing + ", help=" + help + '}'; } } diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java index 347bb62135e9f..25d52c57fb785 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java @@ -93,7 +93,9 @@ public void syncHoodieTable() { // sync a RO table for MOR syncHoodieTable(roTableTableName.get(), false); // sync a RT table for MOR - syncHoodieTable(snapshotTableName, true); + if (!cfg.skipRTSync) { + syncHoodieTable(snapshotTableName, true); + } break; default: LOG.error("Unknown table type " + hoodieDLAClient.getTableType()); diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java index 34a96c9ad8fe6..02c07d6e5861f 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java @@ -70,7 +70,8 @@ public class HoodieDLAClient extends AbstractSyncHoodieClient { private PartitionValueExtractor partitionValueExtractor; public HoodieDLAClient(DLASyncConfig syncConfig, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, fs); + super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, + syncConfig.verifyMetadataFileListing, fs); this.dlaConfig = syncConfig; try { this.partitionValueExtractor = diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java index 636fd0b8b6c87..ad47b71f84948 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java @@ -36,6 +36,7 @@ public class Utils { public static String DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.dla_sync.partition_extractor_class"; public static String DLA_ASSUME_DATE_PARTITIONING = "hoodie.datasource.dla_sync.assume_date_partitioning"; public static String DLA_SKIP_RO_SUFFIX = "hoodie.datasource.dla_sync.skip_ro_suffix"; + public static String DLA_SKIP_RT_SYNC = "hoodie.datasource.dla_sync.skip_rt_sync"; public static String DLA_SYNC_HIVE_STYLE_PARTITIONING = "hoodie.datasource.dla_sync.hive.style.partitioning"; public static Properties configToProperties(DLASyncConfig cfg) { @@ -69,6 +70,7 @@ public static DLASyncConfig propertiesToConfig(Properties properties) { config.partitionValueExtractorClass = properties.getProperty(DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY); config.assumeDatePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_ASSUME_DATE_PARTITIONING, "false")); config.skipROSuffix = Boolean.parseBoolean(properties.getProperty(DLA_SKIP_RO_SUFFIX, "false")); + config.skipRTSync = Boolean.parseBoolean(properties.getProperty(DLA_SKIP_RT_SYNC, "false")); config.useDLASyncHiveStylePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_SYNC_HIVE_STYLE_PARTITIONING, "false")); return config; } diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index d485da459251b..3718046fc94c4 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index fa223980d663e..dd9d483fb10b7 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -18,6 +18,8 @@ package org.apache.hudi.hive; +import org.apache.hudi.common.config.HoodieMetadataConfig; + import com.beust.jcommander.Parameter; import java.io.Serializable; @@ -77,6 +79,12 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") public Boolean skipROSuffix = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @@ -84,6 +92,9 @@ public class HiveSyncConfig implements Serializable { + "Disabled by default for backward compatibility.") public Boolean supportTimestamp = false; + @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") + public Boolean decodePartition = false; + public static HiveSyncConfig copy(HiveSyncConfig cfg) { HiveSyncConfig newConfig = new HiveSyncConfig(); newConfig.basePath = cfg.basePath; @@ -96,16 +107,35 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.jdbcUrl = cfg.jdbcUrl; newConfig.tableName = cfg.tableName; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; + newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; + newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; newConfig.supportTimestamp = cfg.supportTimestamp; + newConfig.decodePartition = cfg.decodePartition; return newConfig; } @Override public String toString() { - return "HiveSyncConfig{databaseName='" + databaseName + '\'' + ", tableName='" + tableName + '\'' - + ", hiveUser='" + hiveUser + '\'' + ", hivePass='" + hivePass + '\'' + ", jdbcUrl='" + jdbcUrl + '\'' - + ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='" - + partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning + '\'' + ", supportTimestamp='" + supportTimestamp + '\'' - + ", usePreApacheInputFormat=" + usePreApacheInputFormat + ", useJdbc=" + useJdbc + ", help=" + help + '}'; + return "HiveSyncConfig{" + + "databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", baseFileFormat='" + baseFileFormat + '\'' + + ", hiveUser='" + hiveUser + '\'' + + ", hivePass='" + hivePass + '\'' + + ", jdbcUrl='" + jdbcUrl + '\'' + + ", basePath='" + basePath + '\'' + + ", partitionFields=" + partitionFields + + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + + ", assumeDatePartitioning=" + assumeDatePartitioning + + ", usePreApacheInputFormat=" + usePreApacheInputFormat + + ", useJdbc=" + useJdbc + + ", autoCreateDatabase=" + autoCreateDatabase + + ", skipROSuffix=" + skipROSuffix + + ", help=" + help + + ", supportTimestamp=" + supportTimestamp + + ", decodePartition=" + decodePartition + + ", useFileListingFromMetadata=" + useFileListingFromMetadata + + ", verifyMetadataFileListing=" + verifyMetadataFileListing + + '}'; } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 6d85395d3f6e5..b6211671b5119 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -18,6 +18,9 @@ package org.apache.hudi.hive; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; @@ -73,7 +76,7 @@ public class HoodieHiveClient extends AbstractSyncHoodieClient { private HiveConf configuration; public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(cfg.basePath, cfg.assumeDatePartitioning, fs); + super(cfg.basePath, cfg.assumeDatePartitioning, cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, fs); this.syncConfig = cfg; this.fs = fs; @@ -162,7 +165,17 @@ private String getPartitionClause(String partition) { + ". Check partition strategy. "); List partBuilder = new ArrayList<>(); for (int i = 0; i < syncConfig.partitionFields.size(); i++) { - partBuilder.add("`" + syncConfig.partitionFields.get(i) + "`='" + partitionValues.get(i) + "'"); + String partitionValue = partitionValues.get(i); + // decode the partition before sync to hive to prevent multiple escapes of HIVE + if (syncConfig.decodePartition) { + try { + // This is a decode operator for encode in KeyGenUtils#getRecordPartitionPath + partitionValue = URLDecoder.decode(partitionValue, StandardCharsets.UTF_8.toString()); + } catch (UnsupportedEncodingException e) { + throw new HoodieHiveSyncException("error in decode partition: " + partitionValue, e); + } + } + partBuilder.add("`" + syncConfig.partitionFields.get(i) + "`='" + partitionValue + "'"); } return String.join(",", partBuilder); } @@ -194,7 +207,6 @@ List getPartitionEvents(List tablePartitions, List paths = new HashMap<>(); for (Partition tablePartition : tablePartitions) { List hivePartitionValues = tablePartition.getValues(); - Collections.sort(hivePartitionValues); String fullTablePartitionPath = Path.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getSd().getLocation())).toUri().getPath(); paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath); @@ -206,7 +218,6 @@ List getPartitionEvents(List tablePartitions, List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); - Collections.sort(storagePartitionValues); if (!storagePartitionValues.isEmpty()) { String storageValue = String.join(", ", storagePartitionValues); if (!paths.containsKey(storageValue)) { diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 1d8cbd85347fd..8a1ea4f893927 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -21,10 +21,10 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.Option; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.hudi.hive.testutils.HiveTestUtil; import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.parquet.schema.MessageType; @@ -56,7 +56,7 @@ private static Stream useJdbc() { } private static Iterable useJdbcAndSchemaFromCommitMetadata() { - return Arrays.asList(new Object[][] { { true, true }, { true, false }, { false, true }, { false, false } }); + return Arrays.asList(new Object[][] {{true, true}, {true, false}, {false, true}, {false, false}}); } @BeforeEach @@ -347,7 +347,7 @@ public void testSyncMergeOnRead(boolean useJdbc, boolean useSchemaFromCommitMeta assertEquals(hiveClient.getTableSchema(roTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + HiveTestUtil.hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), - "Hive Schema should match the table schema + partition field"); + "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. assertEquals(hiveClient.getTableSchema(roTableName).size(), @@ -377,7 +377,7 @@ public void testSyncMergeOnRead(boolean useJdbc, boolean useSchemaFromCommitMeta assertEquals(hiveClient.getTableSchema(roTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + HiveTestUtil.hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), - "Hive Schema should match the evolved table schema + partition field"); + "Hive Schema should match the evolved table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. assertEquals(hiveClient.getTableSchema(roTableName).size(), @@ -418,7 +418,7 @@ public void testSyncMergeOnReadRT(boolean useJdbc, boolean useSchemaFromCommitMe assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + HiveTestUtil.hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), - "Hive Schema should match the table schema + partition field"); + "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), @@ -489,6 +489,50 @@ public void testMultiPartitionKeySync(boolean useJdbc) throws Exception { "Table partitions should match the number of partitions we wrote"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was sycned should be updated in the TBLPROPERTIES"); + + // HoodieHiveClient had a bug where partition vals were sorted + // and stored as keys in a map. The following tests this particular case. + // Now lets create partition "2010/01/02" and followed by "2010/02/01". + String commitTime2 = "101"; + HiveTestUtil.addCOWPartition("2010/01/02", true, true, commitTime2); + + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(instantTime)); + assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit"); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); + assertEquals(1, partitionEvents.size(), "There should be only one paritition event"); + assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD"); + + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + // Sync should add the one partition + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + "Table partitions should match the number of partitions we wrote"); + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + "The last commit that was sycned should be 101"); + + // create partition "2010/02/01" and ensure sync works + String commitTime3 = "102"; + HiveTestUtil.addCOWPartition("2010/02/01", true, true, commitTime3); + HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), + hiveClient.getDataSchema().getColumns().size() + 3, + "Hive Schema should match the table schema + partition fields"); + assertEquals(7, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + "Table partitions should match the number of partitions we wrote"); + assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + "The last commit that was sycned should be updated in the TBLPROPERTIES"); + assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).size()); } @ParameterizedTest @@ -507,17 +551,17 @@ public void testNonPartitionedSync(boolean useJdbc) throws Exception { HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); tool.syncHoodieTable(); assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); + "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), - hiveClient.getDataSchema().getColumns().size(), - "Hive Schema should match the table schema,ignoring the partition fields"); + hiveClient.getDataSchema().getColumns().size(), + "Hive Schema should match the table schema,ignoring the partition fields"); assertEquals(0, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), - "Table should not have partitions because of the NonPartitionedExtractor"); + "Table should not have partitions because of the NonPartitionedExtractor"); } @ParameterizedTest diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index d0d1b667aea20..09090532bf919 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -210,6 +210,14 @@ public static void addCOWPartitions(int numberOfPartitions, boolean isParquetSch createCommitFile(commitMetadata, instantTime); } + public static void addCOWPartition(String partitionPath, boolean isParquetSchemaSimple, + boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException { + HoodieCommitMetadata commitMetadata = + createPartition(partitionPath, isParquetSchemaSimple, useSchemaFromCommitMetadata, instantTime); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + createCommitFile(commitMetadata, instantTime); + } + public static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean isLogSchemaSimple, boolean useSchemaFromCommitMetadata, DateTime startFrom, String instantTime, String deltaCommitTime) throws IOException, URISyntaxException, InterruptedException { @@ -266,6 +274,18 @@ private static HoodieCommitMetadata createPartitions(int numberOfPartitions, boo return commitMetadata; } + private static HoodieCommitMetadata createPartition(String partitionPath, boolean isParquetSchemaSimple, + boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath); + fileSystem.makeQualified(partPath); + fileSystem.mkdirs(partPath); + List writeStats = createTestData(partPath, isParquetSchemaSimple, instantTime); + writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s)); + addSchemaToCommitMetadata(commitMetadata, isParquetSchemaSimple, useSchemaFromCommitMetadata); + return commitMetadata; + } + private static List createTestData(Path partPath, boolean isParquetSchemaSimple, String instantTime) throws IOException, URISyntaxException { List writeStats = new ArrayList<>(); diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index e974d72e1f4f6..153a046210c9a 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java index 419ea16deefd6..8d03252d49bb5 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java @@ -18,8 +18,7 @@ package org.apache.hudi.sync.common; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -27,12 +26,13 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; -import java.io.IOException; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -40,18 +40,25 @@ import java.util.Map; public abstract class AbstractSyncHoodieClient { + private static final Logger LOG = LogManager.getLogger(AbstractSyncHoodieClient.class); + protected final HoodieTableMetaClient metaClient; protected final HoodieTableType tableType; protected final FileSystem fs; private String basePath; private boolean assumeDatePartitioning; + private boolean useFileListingFromMetadata; + private boolean verifyMetadataFileListing; - public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, FileSystem fs) { + public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, boolean useFileListingFromMetadata, + boolean verifyMetadataFileListing, FileSystem fs) { this.metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); this.tableType = metaClient.getTableType(); this.basePath = basePath; this.assumeDatePartitioning = assumeDatePartitioning; + this.useFileListingFromMetadata = useFileListingFromMetadata; + this.verifyMetadataFileListing = verifyMetadataFileListing; this.fs = fs; } @@ -119,11 +126,9 @@ public MessageType getDataSchema() { public List getPartitionsWrittenToSince(Option lastCommitTimeSynced) { if (!lastCommitTimeSynced.isPresent()) { LOG.info("Last commit time synced is not known, listing all partitions in " + basePath + ",FS :" + fs); - try { - return FSUtils.getAllPartitionPaths(fs, basePath, assumeDatePartitioning); - } catch (IOException e) { - throw new HoodieIOException("Failed to list all partitions in " + basePath, e); - } + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + return FSUtils.getAllPartitionPaths(engineContext, basePath, useFileListingFromMetadata, verifyMetadataFileListing, + assumeDatePartitioning); } else { LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline() diff --git a/hudi-sync/pom.xml b/hudi-sync/pom.xml index d056749e6c134..40650037d2475 100644 --- a/hudi-sync/pom.xml +++ b/hudi-sync/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml index f285cd8e5818b..05e422ef6da5e 100644 --- a/hudi-timeline-service/pom.xml +++ b/hudi-timeline-service/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java index e008fc542c413..b3e860af7001e 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java @@ -299,6 +299,21 @@ private void registerFileSlicesAPI() { writeValueAsString(ctx, dtos); }, true)); + app.get(RemoteHoodieTableFileSystemView.ALL_REPLACED_FILEGROUPS_BEFORE, new ViewHandler(ctx -> { + List dtos = sliceHandler.getReplacedFileGroupsBefore( + ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), + ctx.queryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM,""), + ctx.queryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM,"")); + writeValueAsString(ctx, dtos); + }, true)); + + app.get(RemoteHoodieTableFileSystemView.ALL_REPLACED_FILEGROUPS_PARTITION, new ViewHandler(ctx -> { + List dtos = sliceHandler.getAllReplacedFileGroups( + ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), + ctx.queryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM,"")); + writeValueAsString(ctx, dtos); + }, true)); + app.get(RemoteHoodieTableFileSystemView.PENDING_CLUSTERING_FILEGROUPS, new ViewHandler(ctx -> { List dtos = sliceHandler.getFileGroupsInPendingClustering( ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow()); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 969f824a6b644..b1bec621f91c1 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -18,7 +18,9 @@ package org.apache.hudi.timeline.service; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -142,24 +144,28 @@ public void run() throws IOException { } public static FileSystemViewManager buildFileSystemViewManager(Config config, SerializableConfiguration conf) { + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(conf.get()); + // Just use defaults for now + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build(); + switch (config.viewStorageType) { case MEMORY: FileSystemViewStorageConfig.Builder inMemConfBuilder = FileSystemViewStorageConfig.newBuilder(); inMemConfBuilder.withStorageType(FileSystemViewStorageType.MEMORY); - return FileSystemViewManager.createViewManager(conf, inMemConfBuilder.build()); + return FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, inMemConfBuilder.build()); case SPILLABLE_DISK: { FileSystemViewStorageConfig.Builder spillableConfBuilder = FileSystemViewStorageConfig.newBuilder(); spillableConfBuilder.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK) .withBaseStoreDir(config.baseStorePathForFileGroups) .withMaxMemoryForView(config.maxViewMemPerTableInMB * 1024 * 1024L) .withMemFractionForPendingCompaction(config.memFractionForCompactionPerTable); - return FileSystemViewManager.createViewManager(conf, spillableConfBuilder.build()); + return FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, spillableConfBuilder.build()); } case EMBEDDED_KV_STORE: { FileSystemViewStorageConfig.Builder rocksDBConfBuilder = FileSystemViewStorageConfig.newBuilder(); rocksDBConfBuilder.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE) .withRocksDBPath(config.rocksDBPath); - return FileSystemViewManager.createViewManager(conf, rocksDBConfBuilder.build()); + return FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, rocksDBConfBuilder.build()); } default: throw new IllegalArgumentException("Invalid view manager storage type :" + config.viewStorageType); diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java index 18c5eb17ccda7..2180e4ead46de 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java @@ -94,6 +94,16 @@ public List getReplacedFileGroupsBeforeOrOn(String basePath, Strin .collect(Collectors.toList()); } + public List getReplacedFileGroupsBefore(String basePath, String maxCommitTime, String partitionPath) { + return viewManager.getFileSystemView(basePath).getReplacedFileGroupsBefore(maxCommitTime, partitionPath).map(FileGroupDTO::fromFileGroup) + .collect(Collectors.toList()); + } + + public List getAllReplacedFileGroups(String basePath, String partitionPath) { + return viewManager.getFileSystemView(basePath).getAllReplacedFileGroups(partitionPath).map(FileGroupDTO::fromFileGroup) + .collect(Collectors.toList()); + } + public List getFileGroupsInPendingClustering(String basePath) { return viewManager.getFileSystemView(basePath).getFileGroupsInPendingClustering() .map(fgInstant -> ClusteringOpDTO.fromClusteringOp(fgInstant.getLeft(), fgInstant.getRight())) diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java index 9a0ff65bc6090..0865585deff21 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java @@ -18,7 +18,8 @@ package org.apache.hudi.timeline.service.functional; -import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -44,9 +45,12 @@ public class TestRemoteHoodieTableFileSystemView extends TestHoodieTableFileSyst protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) { FileSystemViewStorageConfig sConf = FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build(); + HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().build(); + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + try { server = new TimelineService(0, - FileSystemViewManager.createViewManager(new SerializableConfiguration(metaClient.getHadoopConf()), sConf)); + FileSystemViewManager.createViewManager(localEngineContext, metadataConfig, sConf)); server.startService(); } catch (Exception ex) { throw new RuntimeException(ex); diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml index ab51475de081e..80c30612e1d87 100644 --- a/hudi-utilities/pom.xml +++ b/hudi-utilities/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT 4.0.0 @@ -118,6 +118,11 @@ + + org.apache.hudi + hudi-spark-common + ${project.version} + org.apache.hudi hudi-spark_${scala.binary.version} @@ -129,6 +134,16 @@ + + org.apache.hudi + hudi-spark2_${scala.binary.version} + ${project.version} + + + org.apache.hudi + hudi-spark3_2.12 + ${project.version} + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index af66c17618e25..06245b98aed8d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -20,10 +20,10 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.HoodieJsonPayload; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java new file mode 100644 index 0000000000000..394771caf4a89 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +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 org.jetbrains.annotations.TestOnly; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +public class HoodieClusteringJob { + + private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class); + private final Config cfg; + private transient FileSystem fs; + private TypedProperties props; + private final JavaSparkContext jsc; + + public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) { + this.cfg = cfg; + this.jsc = jsc; + this.props = cfg.propsFilePath == null + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + } + + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { + final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); + + return UtilHelpers + .readConfig(fs, new Path(cfg.propsFilePath), cfg.configs) + .getConfig(); + } + + public static class Config implements Serializable { + @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true) + public String basePath = null; + @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) + public String tableName = null; + @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only need when cluster. " + + "And schedule clustering can generate it.", required = false) + public String clusteringInstantTime = null; + @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) + public int parallelism = 1; + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) + public String sparkMemory = null; + @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) + public int retry = 0; + + @Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering") + public Boolean runSchedule = false; + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for clustering") + public String propsFilePath = null; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", + splitter = IdentitySplitter.class) + public List configs = new ArrayList<>(); + } + + public static void main(String[] args) { + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0 || (!cfg.runSchedule && cfg.clusteringInstantTime == null)) { + cmd.usage(); + System.exit(1); + } + final JavaSparkContext jsc = UtilHelpers.buildSparkContext("clustering-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory); + HoodieClusteringJob clusteringJob = new HoodieClusteringJob(jsc, cfg); + int result = clusteringJob.cluster(cfg.retry); + String resultMsg = String.format("Clustering with basePath: %s, tableName: %s, runSchedule: %s", + cfg.basePath, cfg.tableName, cfg.runSchedule); + if (result == -1) { + LOG.error(resultMsg + " failed"); + } else { + LOG.info(resultMsg + " success"); + } + jsc.stop(); + } + + public int cluster(int retry) { + this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); + int ret = UtilHelpers.retry(retry, () -> { + if (cfg.runSchedule) { + LOG.info("Do schedule"); + Option instantTime = doSchedule(jsc); + int result = instantTime.isPresent() ? 0 : -1; + if (result == 0) { + LOG.info("The schedule instant time is " + instantTime.get()); + } + return result; + } else { + LOG.info("Do cluster"); + return doCluster(jsc); + } + }, "Cluster failed"); + return ret; + } + + private String getSchemaFromLatestInstant() throws Exception { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath, true); + TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient); + if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) { + throw new HoodieException("Cannot run clustering without any completed commits"); + } + Schema schema = schemaUtil.getTableAvroSchema(false); + return schema.toString(); + } + + private int doCluster(JavaSparkContext jsc) throws Exception { + String schemaStr = getSchemaFromLatestInstant(); + SparkRDDWriteClient client = + UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); + JavaRDD writeResponse = + (JavaRDD) client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses(); + return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse); + } + + @TestOnly + public Option doSchedule() throws Exception { + return this.doSchedule(jsc); + } + + private Option doSchedule(JavaSparkContext jsc) throws Exception { + String schemaStr = getSchemaFromLatestInstant(); + SparkRDDWriteClient client = + UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); + return client.scheduleClustering(Option.empty()); + } + +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java index 30d5445b37060..6830f7f7455a0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java @@ -109,21 +109,16 @@ public static void main(String[] args) { public int compact(int retry) { this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); - int ret = -1; - try { - do { - if (cfg.runSchedule) { - if (null == cfg.strategyClassName) { - throw new IllegalArgumentException("Missing Strategy class name for running compaction"); - } - ret = doSchedule(jsc); - } else { - ret = doCompact(jsc); + int ret = UtilHelpers.retry(retry, () -> { + if (cfg.runSchedule) { + if (null == cfg.strategyClassName) { + throw new IllegalArgumentException("Missing Strategy class name for running compaction"); } - } while (ret != 0 && retry-- > 0); - } catch (Throwable t) { - LOG.error(t); - } + return doSchedule(jsc); + } else { + return doCompact(jsc); + } + }, "Compact failed"); return ret; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index 05b46273001c0..ece9b8c93457e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -18,9 +18,10 @@ package org.apache.hudi.utilities; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -68,15 +69,24 @@ static class Config implements Serializable { @Parameter(names = {"--date-partitioned", "-dp"}, description = "Can we assume date partitioning?") boolean shouldAssumeDatePartitioning = false; + + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; } public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir, - final boolean shouldAssumeDatePartitioning) throws IOException { + final boolean shouldAssumeDatePartitioning, + final boolean useFileListingFromMetadata, + final boolean verifyMetadataFileListing) throws IOException { FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration()); final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration()); final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir); final BaseFileOnlyView fsView = new HoodieTableFileSystemView(tableMetadata, tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants()); + HoodieEngineContext context = new HoodieSparkEngineContext(jsc); // Get the latest commit Option latestCommit = tableMetadata.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedInstants().lastInstant(); @@ -88,7 +98,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp)); - List partitions = FSUtils.getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning); + List partitions = FSUtils.getAllPartitionPaths(context, baseDir, useFileListingFromMetadata, verifyMetadataFileListing, shouldAssumeDatePartitioning); if (partitions.size() > 0) { LOG.info(String.format("The job needs to copy %d partitions.", partitions.size())); @@ -99,7 +109,6 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi fs.delete(new Path(outputDir), true); } - HoodieEngineContext context = new HoodieSparkEngineContext(jsc); context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot"); List> filesToCopy = context.flatMap(partitions, partition -> { @@ -111,7 +120,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi // also need to copy over partition metadata Path partitionMetaFile = - new Path(new Path(baseDir, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); + new Path(FSUtils.getPartitionPath(baseDir, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); if (fs1.exists(partitionMetaFile)) { filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); } @@ -122,7 +131,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi context.foreach(filesToCopy, tuple -> { String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); - Path toPartitionPath = new Path(outputDir, partition); + Path toPartitionPath = FSUtils.getPartitionPath(outputDir, partition); FileSystem ifs = FSUtils.getFs(baseDir, serConf.newCopy()); if (!ifs.exists(toPartitionPath)) { @@ -183,7 +192,8 @@ public static void main(String[] args) throws IOException { // Copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning); + copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning, cfg.useFileListingFromMetadata, + cfg.verifyMetadataFileListing); // Stop the job jsc.stop(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index cf69dd2207d80..b9f32cbdca244 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -18,9 +18,9 @@ package org.apache.hudi.utilities; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -117,6 +117,7 @@ public static class Config implements Serializable { public void export(JavaSparkContext jsc, Config cfg) throws IOException { FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, jsc.hadoopConfiguration()); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); if (outputPathExists(fs, cfg)) { throw new HoodieSnapshotExporterException("The target output path already exists."); @@ -128,7 +129,7 @@ public void export(JavaSparkContext jsc, Config cfg) throws IOException { LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp)); - final List partitions = getPartitions(fs, cfg); + final List partitions = getPartitions(engineContext, cfg); if (partitions.isEmpty()) { throw new HoodieSnapshotExporterException("The source dataset has 0 partition to snapshot."); } @@ -153,8 +154,8 @@ private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { return latestCommit.isPresent() ? Option.of(latestCommit.get().getTimestamp()) : Option.empty(); } - private List getPartitions(FileSystem fs, Config cfg) throws IOException { - return FSUtils.getAllPartitionPaths(fs, cfg.sourceBasePath, false); + private List getPartitions(HoodieEngineContext engineContext, Config cfg) { + return FSUtils.getAllPartitionPaths(engineContext, cfg.sourceBasePath, true, false, false); } private void createSuccessTag(FileSystem fs, Config cfg) throws IOException { @@ -208,7 +209,7 @@ private void exportAsHudi(JavaSparkContext jsc, Config cfg, List partiti dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()))); // also need to copy over partition metadata Path partitionMetaFile = - new Path(new Path(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); + new Path(FSUtils.getPartitionPath(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE); FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, serConf.newCopy()); if (fs.exists(partitionMetaFile)) { filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString())); @@ -219,7 +220,7 @@ private void exportAsHudi(JavaSparkContext jsc, Config cfg, List partiti context.foreach(files, tuple -> { String partition = tuple._1(); Path sourceFilePath = new Path(tuple._2()); - Path toPartitionPath = new Path(cfg.targetOutputPath, partition); + Path toPartitionPath = FSUtils.getPartitionPath(cfg.targetOutputPath, partition); FileSystem fs = FSUtils.getFs(cfg.targetOutputPath, serConf.newCopy()); if (!fs.exists(toPartitionPath)) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java index 8974061c4a469..49b93500c0e7c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java @@ -18,8 +18,8 @@ package org.apache.hudi.utilities; -import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.util.ValidationUtils; import com.beust.jcommander.JCommander; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 45171b38845c8..19de67f7b6994 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -37,11 +37,12 @@ import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.schema.DelegatingSchemaProvider; -import org.apache.hudi.utilities.schema.RowBasedSchemaProvider; import org.apache.hudi.utilities.schema.SchemaPostProcessor; import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor; +import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; +import org.apache.hudi.utilities.schema.RowBasedSchemaProvider; import org.apache.hudi.utilities.sources.AvroKafkaSource; import org.apache.hudi.utilities.sources.JsonKafkaSource; import org.apache.hudi.utilities.sources.Source; @@ -54,7 +55,6 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.Accumulator; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -67,6 +67,7 @@ import org.apache.spark.sql.jdbc.JdbcDialect; import org.apache.spark.sql.jdbc.JdbcDialects; import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.LongAccumulator; import java.io.BufferedReader; import java.io.IOException; @@ -291,7 +292,7 @@ public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, Strin } public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD writeResponse) { - Accumulator errors = jsc.accumulator(0); + LongAccumulator errors = jsc.sc().longAccumulator(); writeResponse.foreach(writeStatus -> { if (writeStatus.hasErrors()) { errors.add(1); @@ -400,7 +401,7 @@ public static SchemaProvider getOriginalSchemaProvider(SchemaProvider schemaProv } public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcessor(SchemaProvider provider, - TypedProperties cfg, JavaSparkContext jssc) { + TypedProperties cfg, JavaSparkContext jssc, List transformerClassNames) { if (provider == null) { return null; @@ -409,7 +410,15 @@ public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcesso if (provider instanceof SchemaProviderWithPostProcessor) { return (SchemaProviderWithPostProcessor)provider; } + String schemaPostProcessorClass = cfg.getString(Config.SCHEMA_POST_PROCESSOR_PROP, null); + boolean enableSparkAvroPostProcessor = Boolean.valueOf(cfg.getString(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE, "true")); + + if (transformerClassNames != null && !transformerClassNames.isEmpty() + && enableSparkAvroPostProcessor && StringUtils.isNullOrEmpty(schemaPostProcessorClass)) { + schemaPostProcessorClass = SparkAvroPostProcessor.class.getName(); + } + return new SchemaProviderWithPostProcessor(provider, Option.ofNullable(createSchemaPostProcessor(schemaPostProcessorClass, cfg, jssc))); } @@ -417,6 +426,24 @@ public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcesso public static SchemaProvider createRowBasedSchemaProvider(StructType structType, TypedProperties cfg, JavaSparkContext jssc) { SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType); - return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc); + return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null); } + + @FunctionalInterface + public interface CheckedSupplier { + T get() throws Throwable; + } + + public static int retry(int maxRetryCount, CheckedSupplier supplier, String errorMessage) { + int ret = -1; + try { + do { + ret = supplier.get(); + } while (ret != 0 && maxRetryCount-- > 0); + } catch (Throwable t) { + LOG.error(errorMessage, t); + } + return ret; + } + } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index a27ce996d4bc0..0ce9acaf45c3e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -18,8 +18,8 @@ package org.apache.hudi.utilities.deltastreamer; -import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -41,6 +41,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hive.HiveSyncConfig; @@ -295,7 +296,9 @@ public Pair>> readFromSource( // Retrieve the previous round checkpoints, if any Option resumeCheckpointStr = Option.empty(); if (commitTimelineOpt.isPresent()) { - Option lastCommit = commitTimelineOpt.get().lastInstant(); + // TODO: now not support replace action HUDI-1500 + Option lastCommit = commitTimelineOpt.get() + .filter(instant -> !instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)).lastInstant(); if (lastCommit.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get(), HoodieCommitMetadata.class); @@ -342,7 +345,7 @@ public Pair>> readFromSource( // pass in the schema for the Row-to-Avro conversion // to avoid nullability mismatch between Avro schema and Row schema avroRDDOptional = transformed - .map(t -> AvroConversionUtils.createRdd( + .map(t -> HoodieSparkUtils.createRdd( t, this.userProvidedSchemaProvider.getTargetSchema(), HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()); schemaProvider = this.userProvidedSchemaProvider; @@ -356,7 +359,7 @@ public Pair>> readFromSource( UtilHelpers.createRowBasedSchemaProvider(r.schema(), props, jssc))) .orElse(dataAndCheckpoint.getSchemaProvider()); avroRDDOptional = transformed - .map(t -> AvroConversionUtils.createRdd( + .map(t -> HoodieSparkUtils.createRdd( t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()); } } else { @@ -619,6 +622,8 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName) // Inline compaction is disabled for continuous mode. otherwise enabled for MOR .withInlineCompaction(cfg.isInlineCompactionEnabled()).build()) + .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField) + .build()) .forTable(cfg.targetTableName) .withAutoCommit(autoCommit).withProps(props); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 6d25f4665fe93..10d9453693f6b 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -555,7 +555,7 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config this.props = properties.get(); LOG.info("Creating delta streamer with configs : " + props.toString()); this.schemaProvider = UtilHelpers.wrapSchemaProviderWithPostProcessor( - UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc), props, jssc); + UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc), props, jssc, cfg.transformerClassNames); deltaSync = new DeltaSync(cfg, sparkSession, schemaProvider, props, jssc, fs, conf, this::onInitializingWriteClient); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java index 505deed6d65e4..379cc4b754e2a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java @@ -19,6 +19,7 @@ package org.apache.hudi.utilities.deltastreamer; import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; @@ -73,8 +74,8 @@ public InputBatch> fetchNewDataInAvroFormat(Option allPartitionPaths = FSUtils.getAllPartitionPaths(timelineServer.getFs(), cfg.basePath, true); + JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + List allPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, cfg.basePath, + cfg.useFileListingFromMetadata, cfg.verifyMetadataFileListing, true); Collections.shuffle(allPartitionPaths); List selected = allPartitionPaths.stream().filter(p -> !p.contains("error")).limit(cfg.maxPartitions) .collect(Collectors.toList()); - JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster); + if (!useExternalTimelineServer) { this.timelineServer.startService(); setHostAddrFromSparkConf(jsc.getConf()); @@ -294,6 +297,12 @@ public static class Config implements Serializable { @Parameter(names = {"--wait-for-manual-queries", "-ww"}) public Boolean waitForManualQueries = false; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; + + @Parameter(names = {"--verify-metadata-file-listing"}, description = "Verify file listing from Hudi's metadata against file system") + public Boolean verifyMetadataFileListing = HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE; + @Parameter(names = {"--help", "-h"}) public Boolean help = false; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java index 43f2ff27d2eff..7542755b2491e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java @@ -46,9 +46,9 @@ public static class Config { private final FileSystem fs; - private final Schema sourceSchema; + protected Schema sourceSchema; - private Schema targetSchema; + protected Schema targetSchema; public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { super(props, jssc); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java new file mode 100644 index 0000000000000..9f71a7f11bcd9 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.schema; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaSparkContext; + +/** + * HUDI-1343:Add standard schema postprocessor which would rewrite the schema using spark-avro conversion. + */ +public class SparkAvroPostProcessor extends SchemaPostProcessor { + + public static class Config { + public static final String SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE = + "hoodie.deltastreamer.schemaprovider.spark_avro_post_processor.enable"; + } + + public SparkAvroPostProcessor(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + @Override + public Schema processSchema(Schema schema) { + return AvroConversionUtils.convertStructTypeToAvroSchema( + AvroConversionUtils.convertAvroSchemaToStructType(schema), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME, + RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE); + } +} \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java index 1152cd65cabcd..aed6c6b7358b0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java @@ -52,7 +52,7 @@ public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, Spark @Override protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { Pair, String> selectPathsWithMaxModificationTime = - pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); + pathSelector.getNextFilePathsAndMaxModificationTime(sparkContext, lastCkptStr, sourceLimit); return selectPathsWithMaxModificationTime.getLeft() .map(pathStr -> new InputBatch<>(Option.of(fromFiles(pathStr)), selectPathsWithMaxModificationTime.getRight())) .orElseGet(() -> new InputBatch<>(Option.empty(), selectPathsWithMaxModificationTime.getRight())); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/CsvDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/CsvDFSSource.java index dc40b47dd16cf..1ce50b2627b33 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/CsvDFSSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/CsvDFSSource.java @@ -92,7 +92,7 @@ public CsvDFSSource(TypedProperties props, protected Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { Pair, String> selPathsWithMaxModificationTime = - pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); + pathSelector.getNextFilePathsAndMaxModificationTime(sparkContext, lastCkptStr, sourceLimit); return Pair.of(fromFiles( selPathsWithMaxModificationTime.getLeft()), selPathsWithMaxModificationTime.getRight()); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonDFSSource.java index d34289daa0942..64da4f4f50f5d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonDFSSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonDFSSource.java @@ -44,7 +44,7 @@ public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, Spark @Override protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { Pair, String> selPathsWithMaxModificationTime = - pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); + pathSelector.getNextFilePathsAndMaxModificationTime(sparkContext, lastCkptStr, sourceLimit); return selPathsWithMaxModificationTime.getLeft() .map(pathStr -> new InputBatch<>(Option.of(fromFiles(pathStr)), selPathsWithMaxModificationTime.getRight())) .orElse(new InputBatch<>(Option.empty(), selPathsWithMaxModificationTime.getRight())); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ParquetDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ParquetDFSSource.java index 55d2de2d4c360..a56a878f1fe73 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ParquetDFSSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/ParquetDFSSource.java @@ -45,7 +45,7 @@ public ParquetDFSSource(TypedProperties props, JavaSparkContext sparkContext, Sp @Override public Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { Pair, String> selectPathsWithMaxModificationTime = - pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); + pathSelector.getNextFilePathsAndMaxModificationTime(sparkContext, lastCkptStr, sourceLimit); return selectPathsWithMaxModificationTime.getLeft() .map(pathStr -> Pair.of(Option.of(fromFiles(pathStr)), selectPathsWithMaxModificationTime.getRight())) .orElseGet(() -> Pair.of(Option.empty(), selectPathsWithMaxModificationTime.getRight())); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java index 47419e0297550..d9d3444bf0e97 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java @@ -34,8 +34,10 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; import java.io.IOException; +import java.io.Serializable; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; @@ -43,7 +45,7 @@ import java.util.List; import java.util.stream.Collectors; -public class DFSPathSelector { +public class DFSPathSelector implements Serializable { protected static volatile Logger log = LogManager.getLogger(DFSPathSelector.class); @@ -90,13 +92,26 @@ public static DFSPathSelector createSourceSelector(TypedProperties props, /** * Get the list of files changed since last checkpoint. * + * @param sparkContext JavaSparkContext to help parallelize certain operations * @param lastCheckpointStr the last checkpoint time string, empty if first run * @param sourceLimit max bytes to read each time * @return the list of files concatenated and their latest modified time */ - public Pair, String> getNextFilePathsAndMaxModificationTime(Option lastCheckpointStr, - long sourceLimit) { + public Pair, String> getNextFilePathsAndMaxModificationTime(JavaSparkContext sparkContext, Option lastCheckpointStr, + long sourceLimit) { + return getNextFilePathsAndMaxModificationTime(lastCheckpointStr, sourceLimit); + } + /** + * Get the list of files changed since last checkpoint. + * + * @param lastCheckpointStr the last checkpoint time string, empty if first run + * @param sourceLimit max bytes to read each time + * @return the list of files concatenated and their latest modified time + */ + @Deprecated + public Pair, String> getNextFilePathsAndMaxModificationTime(Option lastCheckpointStr, + long sourceLimit) { try { // obtain all eligible files under root folder. log.info("Root path => " + props.getString(Config.ROOT_INPUT_PATH_PROP) + " source limit => " + sourceLimit); @@ -136,7 +151,7 @@ public Pair, String> getNextFilePathsAndMaxModificationTime(Optio /** * List files recursively, filter out illegible files/directories while doing so. */ - private List listEligibleFiles(FileSystem fs, Path path, long lastCheckpointTime) throws IOException { + protected List listEligibleFiles(FileSystem fs, Path path, long lastCheckpointTime) throws IOException { // skip files/dirs whose names start with (_, ., etc) FileStatus[] statuses = fs.listStatus(path, file -> IGNORE_FILEPREFIX_LIST.stream().noneMatch(pfx -> file.getName().startsWith(pfx))); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java new file mode 100644 index 0000000000000..2cedb6ceaf883 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DatePartitionPathSelector.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.helpers; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.utilities.sources.helpers.DFSPathSelector.Config.ROOT_INPUT_PATH_PROP; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.DATE_PARTITION_DEPTH; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.DEFAULT_DATE_PARTITION_DEPTH; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.DEFAULT_LOOKBACK_DAYS; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.DEFAULT_PARTITIONS_LIST_PARALLELISM; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.LOOKBACK_DAYS; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.PARTITIONS_LIST_PARALLELISM; + +/** + * Custom dfs path selector used to list just the last few days provided there is a date based + * partition. + * + *

This is useful for workloads where there are multiple partition fields and only recent + * partitions are affected by new writes. Especially if the data sits in S3, listing all historical + * data can be time expensive and unnecessary for the above type of workload. + * + *

The date based partition is expected to be of the format '=yyyy-mm-dd' or + * 'yyyy-mm-dd'. The date partition can be at any level. For ex. the partition path can be of the + * form `////` or + * `/</` + */ +public class DatePartitionPathSelector extends DFSPathSelector { + + private static volatile Logger LOG = LogManager.getLogger(DatePartitionPathSelector.class); + + private final int datePartitionDepth; + private final int numPrevDaysToList; + private final LocalDate fromDate; + private final LocalDate currentDate; + private final int partitionsListParallelism; + + /** Configs supported. */ + public static class Config { + public static final String DATE_PARTITION_DEPTH = + "hoodie.deltastreamer.source.dfs.datepartitioned.selector.depth"; + public static final int DEFAULT_DATE_PARTITION_DEPTH = 0; // Implies no (date) partition + + public static final String LOOKBACK_DAYS = + "hoodie.deltastreamer.source.dfs.datepartitioned.selector.lookback.days"; + public static final int DEFAULT_LOOKBACK_DAYS = 2; + + public static final String CURRENT_DATE = + "hoodie.deltastreamer.source.dfs.datepartitioned.selector.currentdate"; + + + public static final String PARTITIONS_LIST_PARALLELISM = + "hoodie.deltastreamer.source.dfs.datepartitioned.selector.parallelism"; + public static final int DEFAULT_PARTITIONS_LIST_PARALLELISM = 20; + } + + public DatePartitionPathSelector(TypedProperties props, Configuration hadoopConf) { + super(props, hadoopConf); + /* + * datePartitionDepth = 0 is same as basepath and there is no partition. In which case + * this path selector would be a no-op and lists all paths under the table basepath. + */ + datePartitionDepth = props.getInteger(DATE_PARTITION_DEPTH, DEFAULT_DATE_PARTITION_DEPTH); + // If not specified the current date is assumed by default. + currentDate = LocalDate.parse(props.getString(Config.CURRENT_DATE, LocalDate.now().toString())); + numPrevDaysToList = props.getInteger(LOOKBACK_DAYS, DEFAULT_LOOKBACK_DAYS); + fromDate = currentDate.minusDays(numPrevDaysToList); + partitionsListParallelism = props.getInteger(PARTITIONS_LIST_PARALLELISM, DEFAULT_PARTITIONS_LIST_PARALLELISM); + } + + @Override + public Pair, String> getNextFilePathsAndMaxModificationTime(JavaSparkContext sparkContext, + Option lastCheckpointStr, + long sourceLimit) { + // obtain all eligible files under root folder. + LOG.info( + "Root path => " + + props.getString(ROOT_INPUT_PATH_PROP) + + " source limit => " + + sourceLimit + + " depth of day partition => " + + datePartitionDepth + + " num prev days to list => " + + numPrevDaysToList + + " from current date => " + + currentDate); + long lastCheckpointTime = lastCheckpointStr.map(Long::parseLong).orElse(Long.MIN_VALUE); + HoodieSparkEngineContext context = new HoodieSparkEngineContext(sparkContext); + SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf()); + List prunedParitionPaths = pruneDatePartitionPaths(context, fs, props.getString(ROOT_INPUT_PATH_PROP)); + + List eligibleFiles = context.flatMap(prunedParitionPaths, + path -> { + FileSystem fs = new Path(path).getFileSystem(serializedConf.get()); + return listEligibleFiles(fs, new Path(path), lastCheckpointTime).stream(); + }, partitionsListParallelism); + // sort them by modification time. + eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime)); + + // Filter based on checkpoint & input size, if needed + long currentBytes = 0; + long maxModificationTime = Long.MIN_VALUE; + List filteredFiles = new ArrayList<>(); + for (FileStatus f : eligibleFiles) { + if (currentBytes + f.getLen() >= sourceLimit) { + // we have enough data, we are done + break; + } + + maxModificationTime = f.getModificationTime(); + currentBytes += f.getLen(); + filteredFiles.add(f); + } + + // no data to read + if (filteredFiles.isEmpty()) { + return new ImmutablePair<>( + Option.empty(), lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE))); + } + + // read the files out. + String pathStr = filteredFiles.stream().map(f -> f.getPath().toString()).collect(Collectors.joining(",")); + + return new ImmutablePair<>(Option.ofNullable(pathStr), String.valueOf(maxModificationTime)); + } + + /** + * Prunes date level partitions to last few days configured by 'NUM_PREV_DAYS_TO_LIST' from + * 'CURRENT_DATE'. Parallelizes listing by leveraging HoodieSparkEngineContext's methods. + */ + public List pruneDatePartitionPaths(HoodieSparkEngineContext context, FileSystem fs, String rootPath) { + List partitionPaths = new ArrayList<>(); + // get all partition paths before date partition level + partitionPaths.add(rootPath); + if (datePartitionDepth <= 0) { + return partitionPaths; + } + SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf()); + for (int i = 0; i < datePartitionDepth; i++) { + partitionPaths = context.flatMap(partitionPaths, path -> { + Path subDir = new Path(path); + FileSystem fileSystem = subDir.getFileSystem(serializedConf.get()); + // skip files/dirs whose names start with (_, ., etc) + FileStatus[] statuses = fileSystem.listStatus(subDir, + file -> IGNORE_FILEPREFIX_LIST.stream().noneMatch(pfx -> file.getName().startsWith(pfx))); + List res = new ArrayList<>(); + for (FileStatus status : statuses) { + res.add(status.getPath().toString()); + } + return res.stream(); + }, partitionsListParallelism); + } + + // Prune date partitions to last few days + return context.getJavaSparkContext().parallelize(partitionPaths, partitionsListParallelism) + .filter(s -> { + String[] splits = s.split("/"); + String datePartition = splits[splits.length - 1]; + LocalDate partitionDate; + if (datePartition.contains("=")) { + String[] moreSplit = datePartition.split("="); + ValidationUtils.checkArgument( + moreSplit.length == 2, + "Partition Field (" + datePartition + ") not in expected format"); + partitionDate = LocalDate.parse(moreSplit[1]); + } else { + partitionDate = LocalDate.parse(datePartition); + } + return (partitionDate.isEqual(fromDate) || partitionDate.isAfter(fromDate)) + && (partitionDate.isEqual(currentDate) || partitionDate.isBefore(currentDate)); + }).collect(); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/Transformer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/Transformer.java index 8d653d84d9f9c..985485e925dbc 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/Transformer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/Transformer.java @@ -44,5 +44,5 @@ public interface Transformer { * @return Transformed Dataset */ @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) - Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset rowDataset, TypedProperties properties); + Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset rowDataset, TypedProperties properties); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java new file mode 100644 index 0000000000000..90c0149b0906c --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities; + +import org.apache.avro.Schema; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.spark.api.java.JavaSparkContext; + +public class SparkAvroSchemaProvider extends SchemaProvider { + + public SparkAvroSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + @Override + public Schema getSourceSchema() { + return new Schema.Parser().parse("{\"type\":\"record\",\"name\":\"test\",\"fields\":[{\"name\": \"day\", \"type\":\"string\"}]}"); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java index 39b84571ebc63..725743dd234b4 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java @@ -22,15 +22,20 @@ import org.apache.hudi.utilities.schema.SchemaPostProcessor; import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config; import org.apache.hudi.utilities.schema.SchemaProvider; +import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.avro.Schema; import org.apache.avro.Schema.Type; import org.apache.avro.SchemaBuilder; + +import org.apache.hudi.utilities.transform.FlatteningTransformer; import org.apache.spark.api.java.JavaSparkContext; import org.junit.jupiter.api.Test; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -39,13 +44,19 @@ public class TestSchemaPostProcessor extends UtilitiesTestBase { private TypedProperties properties = new TypedProperties(); + private static String ORIGINAL_SCHEMA = "{\"name\":\"t3_biz_operation_t_driver\",\"type\":\"record\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"null\",\"string\"],\"default\":null}," + + "{\"name\":\"ums_ts_\",\"type\":[\"null\",\"string\"],\"default\":null}]}"; + + private static String RESULT_SCHEMA = "{\"type\":\"record\",\"name\":\"hoodie_source\",\"namespace\":\"hoodie.source\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"ums_ts_\",\"type\":[\"string\",\"null\"]}]}"; + @Test public void testPostProcessor() throws IOException { properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, DummySchemaPostProcessor.class.getName()); SchemaProvider provider = UtilHelpers.wrapSchemaProviderWithPostProcessor( UtilHelpers.createSchemaProvider(DummySchemaProvider.class.getName(), properties, jsc), - properties, jsc); + properties, jsc,null); Schema schema = provider.getSourceSchema(); assertEquals(schema.getType(), Type.RECORD); @@ -53,6 +64,31 @@ public void testPostProcessor() throws IOException { assertNotNull(schema.getField("testString")); } + @Test + public void testSparkAvro() throws IOException { + properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, SparkAvroPostProcessor.class.getName()); + List transformerClassNames = new ArrayList<>(); + transformerClassNames.add(FlatteningTransformer.class.getName()); + + SchemaProvider provider = + UtilHelpers.wrapSchemaProviderWithPostProcessor( + UtilHelpers.createSchemaProvider(SparkAvroSchemaProvider.class.getName(), properties, jsc), + properties, jsc, transformerClassNames); + + Schema schema = provider.getSourceSchema(); + assertEquals(schema.getType(), Type.RECORD); + assertEquals(schema.getName(), "hoodie_source"); + assertEquals(schema.getNamespace(), "hoodie.source"); + assertNotNull(schema.getField("day")); + } + + @Test + public void testSparkAvroSchema() throws IOException { + SparkAvroPostProcessor processor = new SparkAvroPostProcessor(properties, null); + Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); + assertEquals(processor.processSchema(schema).toString(), RESULT_SCHEMA); + } + public static class DummySchemaPostProcessor extends SchemaPostProcessor { public DummySchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 9b0097e8ba257..eeef8ed791413 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; @@ -41,6 +42,7 @@ import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.utilities.DummySchemaProvider; +import org.apache.hudi.utilities.HoodieClusteringJob; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.sources.CsvDFSSource; @@ -161,6 +163,7 @@ public static void initClass() throws Exception { UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/invalid_hive_sync_uber_config.properties", dfs, dfsBasePath + "/config/invalid_hive_sync_uber_config.properties"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/uber_config.properties", dfs, dfsBasePath + "/config/uber_config.properties"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/short_trip_uber_config.properties", dfs, dfsBasePath + "/config/short_trip_uber_config.properties"); + UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/clusteringjob.properties", dfs, dfsBasePath + "/clusteringjob.properties"); TypedProperties props = new TypedProperties(); props.setProperty("include", "sql-transformer.properties"); @@ -403,6 +406,14 @@ static void waitTillCondition(Function condition, long timeout }); res.get(timeoutInSecs, TimeUnit.SECONDS); } + + static void assertAtLeastNCommits(int minExpected, String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = new HoodieTableMetaClient(fs.getConf(), tablePath); + HoodieTimeline timeline = meta.getActiveTimeline().filterCompletedInstants(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numDeltaCommits = (int) timeline.getInstants().count(); + assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); + } } @Test @@ -622,7 +633,6 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; - // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); cfg.continuousMode = true; @@ -630,15 +640,7 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); - Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { - try { - ds.sync(); - } catch (Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); - } - }); - - TestHelpers.waitTillCondition((r) -> { + deltaStreamerTestRunner(ds, cfg, (r) -> { if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { TestHelpers.assertAtleastNDeltaCommits(5, tableBasePath, dfs); TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs); @@ -648,11 +650,105 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); return true; - }, 180); + }); + } + + private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { + Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { + try { + ds.sync(); + } catch (Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + }); + + TestHelpers.waitTillCondition(condition, 240); ds.shutdownGracefully(); dsFuture.get(); } + @Test + public void testInlineClustering() throws Exception { + String tableBasePath = dfsBasePath + "/inlineClustering"; + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + cfg.continuousMode = true; + cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); + cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true")); + cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP, "2")); + HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); + deltaStreamerTestRunner(ds, cfg, (r) -> { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true); + int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length; + int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length; + LOG.info("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize); + return completeReplaceSize > 0; + }); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true); + assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length); + } + + private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePath, + String clusteringInstantTime, boolean runSchedule) { + HoodieClusteringJob.Config config = new HoodieClusteringJob.Config(); + config.basePath = basePath; + config.clusteringInstantTime = clusteringInstantTime; + config.runSchedule = runSchedule; + config.propsFilePath = dfsBasePath + "/clusteringjob.properties"; + return config; + } + + @Test + public void testHoodieAsyncClusteringJob() throws Exception { + String tableBasePath = dfsBasePath + "/asyncClustering"; + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + cfg.continuousMode = true; + cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); + cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + cfg.configs.add(String.format("%s=true", HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY)); + HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); + deltaStreamerTestRunner(ds, cfg, (r) -> { + TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); + HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, + null, true); + HoodieClusteringJob scheduleClusteringJob = new HoodieClusteringJob(jsc, scheduleClusteringConfig); + Option scheduleClusteringInstantTime = Option.empty(); + try { + scheduleClusteringInstantTime = scheduleClusteringJob.doSchedule(); + } catch (Exception e) { + LOG.warn("Schedule clustering failed", e); + return false; + } + if (scheduleClusteringInstantTime.isPresent()) { + LOG.info("Schedule clustering success, now cluster with instant time " + scheduleClusteringInstantTime.get()); + HoodieClusteringJob.Config clusterClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, + scheduleClusteringInstantTime.get(), false); + HoodieClusteringJob clusterClusteringJob = new HoodieClusteringJob(jsc, clusterClusteringConfig); + clusterClusteringJob.cluster(clusterClusteringConfig.retry); + LOG.info("Cluster success"); + } else { + LOG.warn("Schedule clustering failed"); + } + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true); + int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length; + int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length; + System.out.println("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize); + return completeReplaceSize > 0; + }); + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true); + assertEquals(1, metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length); + } + /** * Test Bulk Insert and upserts with hive syncing. Tests Hudi incremental processing using a 2 step pipeline The first * step involves using a SQL template to transform a source TEST-DATA-SOURCE ============================> HUDI TABLE diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java index 95af888fc48eb..0b19fa0801397 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotCopier.java @@ -18,6 +18,7 @@ package org.apache.hudi.utilities.functional; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -67,7 +68,9 @@ public void testEmptySnapshotCopy() throws IOException { // Do the snapshot HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, true); + copier.snapshot(jsc(), basePath, outputPath, true, + HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE); // Nothing changed; we just bail out assertEquals(fs.listStatus(new Path(basePath)).length, 1); @@ -120,7 +123,8 @@ public void testSnapshotCopy() throws Exception { // Do a snapshot copy HoodieSnapshotCopier copier = new HoodieSnapshotCopier(); - copier.snapshot(jsc(), basePath, outputPath, false); + copier.snapshot(jsc(), basePath, outputPath, false, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE); // Check results assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName()))); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestDatePartitionPathSelector.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestDatePartitionPathSelector.java new file mode 100644 index 0000000000000..b7e127924f998 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestDatePartitionPathSelector.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.helpers; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.utilities.testutils.UtilitiesTestBase; + +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.apache.hudi.utilities.sources.helpers.DFSPathSelector.Config.ROOT_INPUT_PATH_PROP; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.CURRENT_DATE; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.DATE_PARTITION_DEPTH; +import static org.apache.hudi.utilities.sources.helpers.DatePartitionPathSelector.Config.LOOKBACK_DAYS; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestDatePartitionPathSelector extends HoodieClientTestHarness { + + private transient HoodieSparkEngineContext context = null; + static List totalDates; + + @BeforeAll + public static void initClass() { + String s = "2020-07-21"; + String e = "2020-07-25"; + LocalDate start = LocalDate.parse(s); + LocalDate end = LocalDate.parse(e); + totalDates = new ArrayList<>(); + while (!start.isAfter(end)) { + totalDates.add(start); + start = start.plusDays(1); + } + } + + @BeforeEach + public void setup() { + initSparkContexts(); + initPath(); + initFileSystem(); + context = new HoodieSparkEngineContext(jsc); + } + + @AfterEach + public void teardown() throws Exception { + cleanupResources(); + } + + /* + * Create Date partitions with some files under each of the leaf Dirs. + */ + public List createDatePartitionsWithFiles(List leafDirs, boolean hiveStyle) + throws IOException { + List allFiles = new ArrayList<>(); + for (Path path : leafDirs) { + List datePartitions = generateDatePartitionsUnder(path, hiveStyle); + for (Path datePartition : datePartitions) { + allFiles.addAll(createRandomFilesUnder(datePartition)); + } + } + return allFiles; + } + + /** + * Create all parent level dirs before the date partitions. + * + * @param root Current parent dir. Initially this points to table basepath. + * @param dirs List o sub dirs to be created under root. + * @param depth Depth of partitions before date partitions. + * @param leafDirs Collect list of leaf dirs. These will be the immediate parents of date based partitions. + * @throws IOException + */ + public void createParentDirsBeforeDatePartitions(Path root, List dirs, int depth, List leafDirs) + throws IOException { + if (depth <= 0) { + leafDirs.add(root); + return; + } + for (String s : dirs) { + Path subdir = new Path(root, s); + fs.mkdirs(subdir); + createParentDirsBeforeDatePartitions(subdir, generateRandomStrings(), depth - 1, leafDirs); + } + } + + /* + * Random string generation util used for generating file names or file contents. + */ + private List generateRandomStrings() { + List subDirs = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + subDirs.add(UUID.randomUUID().toString()); + } + return subDirs; + } + + /* + * Generate date based partitions under a parent dir with or without hivestyle formatting. + */ + private List generateDatePartitionsUnder(Path parent, boolean hiveStyle) throws IOException { + List datePartitions = new ArrayList<>(); + String prefix = (hiveStyle ? "dt=" : ""); + for (int i = 0; i < 5; i++) { + Path child = new Path(parent, prefix + totalDates.get(i).toString()); + fs.mkdirs(child); + datePartitions.add(child); + } + return datePartitions; + } + + /* + * Creates random files under the given directory. + */ + private List createRandomFilesUnder(Path path) throws IOException { + List resultFiles = new ArrayList<>(); + List fileNames = generateRandomStrings(); + for (String fileName : fileNames) { + List fileContent = generateRandomStrings(); + String[] lines = new String[fileContent.size()]; + lines = fileContent.toArray(lines); + Path file = new Path(path, fileName); + UtilitiesTestBase.Helpers.saveStringsToDFS(lines, fs, file.toString()); + resultFiles.add(file); + } + return resultFiles; + } + + private static TypedProperties getProps( + String basePath, int datePartitionDepth, int numDaysToList, String currentDate) { + TypedProperties properties = new TypedProperties(); + properties.put(ROOT_INPUT_PATH_PROP, basePath); + properties.put(DATE_PARTITION_DEPTH, "" + datePartitionDepth); + properties.put(LOOKBACK_DAYS, "" + numDaysToList); + properties.put(CURRENT_DATE, currentDate); + return properties; + } + + /* + * Return test params => (table basepath, date partition's depth, + * num of prev days to list, current date, is date partition formatted in hive style?, + * expected number of paths after pruning) + */ + private static Stream configParams() { + Object[][] data = + new Object[][] { + {"table1", 0, 2, "2020-07-25", true, 1}, + {"table2", 0, 2, "2020-07-25", false, 1}, + {"table3", 1, 3, "2020-07-25", true, 4}, + {"table4", 1, 3, "2020-07-25", false, 4}, + {"table5", 2, 1, "2020-07-25", true, 10}, + {"table6", 2, 1, "2020-07-25", false, 10}, + {"table7", 3, 2, "2020-07-25", true, 75}, + {"table8", 3, 2, "2020-07-25", false, 75} + }; + return Stream.of(data).map(Arguments::of); + } + + @ParameterizedTest(name = "[{index}] {0}") + @MethodSource("configParams") + public void testPruneDatePartitionPaths( + String tableName, + int datePartitionDepth, + int numPrevDaysToList, + String currentDate, + boolean isHiveStylePartition, + int expectedNumFiles) + throws IOException { + TypedProperties props = getProps(basePath + "/" + tableName, datePartitionDepth, numPrevDaysToList, currentDate); + DatePartitionPathSelector pathSelector = new DatePartitionPathSelector(props, jsc.hadoopConfiguration()); + + Path root = new Path(props.getString(ROOT_INPUT_PATH_PROP)); + int totalDepthBeforeDatePartitions = props.getInteger(DATE_PARTITION_DEPTH) - 1; + + // Create parent dir + List leafDirs = new ArrayList<>(); + createParentDirsBeforeDatePartitions(root, generateRandomStrings(), totalDepthBeforeDatePartitions, leafDirs); + createDatePartitionsWithFiles(leafDirs, isHiveStylePartition); + + List paths = pathSelector.pruneDatePartitionPaths(context, fs, root.toString()); + + assertEquals(expectedNumFiles, pathSelector.pruneDatePartitionPaths(context, fs, root.toString()).size()); + } +} diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/clusteringjob.properties b/hudi-utilities/src/test/resources/delta-streamer-config/clusteringjob.properties new file mode 100644 index 0000000000000..9a089fe41ad20 --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/clusteringjob.properties @@ -0,0 +1,18 @@ +### +# 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. +### +hoodie.clustering.inline.max.commits=2 \ No newline at end of file diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 80e828bf48d72..b8a8cd1efab37 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -17,12 +17,11 @@ limitations under the License. --> - + hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -51,7 +50,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index b12da4cc3c544..215048b958299 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -48,7 +48,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index 7cbf33288d6ea..d310cc8baae2f 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -48,7 +48,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 84285dcdc534b..f11e95d6e6e49 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -13,12 +13,11 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -47,7 +46,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml @@ -73,6 +72,8 @@ org.apache.hudi:hudi-spark-client org.apache.hudi:hudi-utilities_${scala.binary.version} org.apache.hudi:hudi-spark_${scala.binary.version} + org.apache.hudi:hudi-spark2_${scala.binary.version} + org.apache.hudi:hudi-spark3_2.12 org.apache.hudi:hudi-hive-sync org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr @@ -339,6 +340,18 @@ ${project.version} + + org.apache.hudi + hudi-spark2_${scala.binary.version} + ${project.version} + + + + org.apache.hudi + hudi-spark3_2.12 + ${project.version} + + org.apache.hadoop hadoop-hdfs diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index eb50882de9480..4217427a8c9d1 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -48,7 +48,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml @@ -206,7 +206,7 @@ presto-shade-unbundle-bootstrap provided - + diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index e4f4bbceddc54..19184bf5881a2 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -48,11 +48,11 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml - + true @@ -66,7 +66,10 @@ org.apache.hudi:hudi-common org.apache.hudi:hudi-client-common org.apache.hudi:hudi-spark-client + org.apache.hudi:hudi-spark-common org.apache.hudi:hudi-spark_${scala.binary.version} + org.apache.hudi:hudi-spark2_${scala.binary.version} + org.apache.hudi:hudi-spark3_2.12 org.apache.hudi:hudi-hive-sync org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr @@ -156,6 +159,10 @@ com.codahale.metrics. org.apache.hudi.com.codahale.metrics. + + org.apache.commons.codec. + org.apache.hudi.org.apache.commons.codec. + org.eclipse.jetty. org.apache.hudi.org.apache.jetty. @@ -220,11 +227,26 @@ hudi-hive-sync ${project.version} + + org.apache.hudi + hudi-spark-common + ${project.version} + org.apache.hudi hudi-spark_${scala.binary.version} ${project.version} + + org.apache.hudi + hudi-spark2_${scala.binary.version} + ${project.version} + + + org.apache.hudi + hudi-spark3_2.12 + ${project.version} + org.apache.hudi hudi-timeline-service @@ -344,7 +366,7 @@ spark-shade-unbundle-avro provided - + diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 4a03369015407..74deec05e2828 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -17,13 +17,11 @@ limitations under the License. --> - + hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 39e48bbb50ba4..20ebe76e3ebaa 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -15,12 +15,11 @@ See the License for the specific language governing permissions and limitations under the License. --> - + hudi org.apache.hudi - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT ../../pom.xml 4.0.0 @@ -49,7 +48,7 @@ shade - true + ${shadeSources} ${project.build.directory}/dependency-reduced-pom.xml @@ -69,7 +68,10 @@ org.apache.hudi:hudi-client-common org.apache.hudi:hudi-spark-client org.apache.hudi:hudi-utilities_${scala.binary.version} + org.apache.hudi:hudi-spark-common org.apache.hudi:hudi-spark_${scala.binary.version} + org.apache.hudi:hudi-spark2_${scala.binary.version} + org.apache.hudi:hudi-spark3_2.12 org.apache.hudi:hudi-hive-sync org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr @@ -105,6 +107,7 @@ io.prometheus:simpleclient_common com.yammer.metrics:metrics-core org.apache.spark:spark-streaming-kafka-0-10_${scala.binary.version} + org.apache.spark:spark-token-provider-kafka-0-10_${scala.binary.version} org.apache.kafka:kafka_${scala.binary.version} com.101tec:zkclient org.apache.kafka:kafka-clients @@ -227,11 +230,26 @@ + + org.apache.hudi + hudi-spark-common + ${project.version} + org.apache.hudi hudi-spark_${scala.binary.version} ${project.version} + + org.apache.hudi + hudi-spark2_${scala.binary.version} + ${project.version} + + + org.apache.hudi + hudi-spark3_2.12 + ${project.version} + org.apache.hudi hudi-utilities_${scala.binary.version} diff --git a/pom.xml b/pom.xml index 80876cf11e2cc..09145a5775a64 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ org.apache.hudi hudi pom - 0.6.1-SNAPSHOT + 0.8.0-SNAPSHOT Apache Hudi brings stream style processing on big data https://github.com/apache/hudi Hudi @@ -39,7 +39,7 @@ hudi-cli hudi-client hudi-hadoop-mr - hudi-spark + hudi-spark-datasource hudi-timeline-service hudi-utilities hudi-sync @@ -84,6 +84,10 @@ 1.8 2.6.7 + 2.6.7.3 + 2.6.7.1 + 2.7.4 + 2.10.0 2.0.0 2.17 1.10.1 @@ -101,10 +105,14 @@ 4.1.1 0.8.0 4.4.1 - 2.4.4 + ${spark2.version} 1.11.2 + 2.4.4 + 3.0.0 1.8.2 - 2.11.12 + 2.11.12 + 2.12.10 + ${scala11.version} 2.11 0.12 3.3.1 @@ -120,18 +128,20 @@ ${skipTests} ${skipTests} ${skipTests} + ${skipTests} UTF-8 ${project.basedir} provided - + compile org.apache.hudi.spark. provided - + -Xmx2g 0.8.5 compile org.apache.hudi. + true @@ -204,7 +214,7 @@ - + @@ -432,7 +442,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.version}.3 + ${fasterxml.jackson.databind.version} com.fasterxml.jackson.datatype @@ -442,7 +452,7 @@ com.fasterxml.jackson.module jackson-module-scala_${scala.binary.version} - ${fasterxml.version}.1 + ${fasterxml.jackson.module.scala.version} @@ -940,14 +950,6 @@ confluent https://packages.confluent.io/maven/ - - libs-milestone - https://repo.spring.io/libs-milestone/ - - - libs-release - https://repo.spring.io/libs-release/ - @@ -1306,7 +1308,7 @@ scala-2.12 - 2.12.10 + ${scala12.version} 2.12 @@ -1341,6 +1343,38 @@ + + + spark3 + + ${spark3.version} + ${scala12.version} + 2.12 + 2.4.1 + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + true + + + + spark3 + + + + + + skipShadeSources + + false + + + + skipShadeSources + + + diff --git a/scripts/report_coverage.sh b/scripts/report_coverage.sh index e6f933920d897..79bbce00f7086 100755 --- a/scripts/report_coverage.sh +++ b/scripts/report_coverage.sh @@ -21,10 +21,11 @@ modules=( hudi-client hudi-common hudi-examples + hudi-flink hudi-hadoop-mr - hudi-hive-sync hudi-integ-test - hudi-spark + hudi-spark-datasource + hudi-sync hudi-timeline-service hudi-utilities ) diff --git a/style/checkstyle.xml b/style/checkstyle.xml index c816abe995eb9..7dbce7973bfde 100644 --- a/style/checkstyle.xml +++ b/style/checkstyle.xml @@ -304,5 +304,10 @@ + + + + + diff --git a/style/import-control.xml b/style/import-control.xml new file mode 100644 index 0000000000000..cd35792cb40d9 --- /dev/null +++ b/style/import-control.xml @@ -0,0 +1,26 @@ + + + + + + + + +