From 1ba82206176b10c0c7392225de75756fcde70192 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Sun, 13 Mar 2022 18:11:37 -0700 Subject: [PATCH 01/33] [HUDI-3613] Adding/fixing yamls for metadata (#5029) --- .../test-suite/cow-spark-long-running.yaml | 2 - ...er-long-running-multi-partitions-hive.yaml | 2 - ...ong-running-multi-partitions-metadata.yaml | 76 +++++++++++++++++++ ...treamer-long-running-multi-partitions.yaml | 2 - ...reamer-medium-full-dataset-validation.yaml | 2 - 5 files changed, 76 insertions(+), 8 deletions(-) create mode 100644 docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml diff --git a/docker/demo/config/test-suite/cow-spark-long-running.yaml b/docker/demo/config/test-suite/cow-spark-long-running.yaml index 8a1e58f840a37..8229ba749502e 100644 --- a/docker/demo/config/test-suite/cow-spark-long-running.yaml +++ b/docker/demo/config/test-suite/cow-spark-long-running.yaml @@ -67,7 +67,5 @@ dag_content: last_validate: config: execute_itr_count: 30 - validate_clean: true - validate_archival: true type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml index 324a4b4a6d0d5..68d14a02dc54b 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml @@ -83,7 +83,5 @@ dag_content: last_validate: config: execute_itr_count: 50 - validate_clean: true - validate_archival: true type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml new file mode 100644 index 0000000000000..817552224e8f3 --- /dev/null +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml @@ -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. +dag_name: deltastreamer-long-running-multi-partitions.yaml +dag_rounds: 30 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 1000 + num_partitions_insert: 50 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 1000 + num_partitions_insert: 2 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 2 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: third_insert + first_delete: + config: + num_partitions_delete: 50 + num_records_delete: 8000 + type: DeleteNode + deps: first_upsert + second_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_delete + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: second_hive_sync + last_validate: + config: + execute_itr_count: 30 + type: ValidateAsyncOperations + deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml index 9d2766f1a5a7e..ba490a8c194dd 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml @@ -72,7 +72,5 @@ dag_content: last_validate: config: execute_itr_count: 50 - validate_clean: true - validate_archival: true type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml index 2fc4961e15c07..aa83b2780127c 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml @@ -75,7 +75,5 @@ dag_content: last_validate: config: execute_itr_count: 20 - validate_clean: true - validate_archival: true type: ValidateAsyncOperations deps: second_validate From 465d553df8c2b157a49e837e1a91cee46ba889d4 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Mon, 14 Mar 2022 14:22:07 +0800 Subject: [PATCH 02/33] [HUDI-3600] Tweak the default cleaning strategy to be more streaming friendly for flink (#5010) --- .../hudi/configuration/FlinkOptions.java | 12 +++++----- .../hudi/table/TestHoodieTableFactory.java | 22 ++++++++++--------- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 1be90603605cd..34bab1285480c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -552,21 +552,21 @@ private FlinkOptions() { public static final ConfigOption CLEAN_RETAIN_COMMITS = ConfigOptions .key("clean.retain_commits") .intType() - .defaultValue(10)// default 10 commits + .defaultValue(30)// default 30 commits .withDescription("Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n" - + "This also directly translates into how much you can incrementally pull on this table, default 10"); + + "This also directly translates into how much you can incrementally pull on this table, default 30"); public static final ConfigOption ARCHIVE_MAX_COMMITS = ConfigOptions .key("archive.max_commits") .intType() - .defaultValue(30)// default max 30 commits - .withDescription("Max number of commits to keep before archiving older commits into a sequential log, default 30"); + .defaultValue(50)// default max 50 commits + .withDescription("Max number of commits to keep before archiving older commits into a sequential log, default 50"); public static final ConfigOption ARCHIVE_MIN_COMMITS = ConfigOptions .key("archive.min_commits") .intType() - .defaultValue(20)// default min 20 commits - .withDescription("Min number of commits to keep before archiving older commits into a sequential log, default 20"); + .defaultValue(40)// default min 40 commits + .withDescription("Min number of commits to keep before archiving older commits into a sequential log, default 40"); // ------------------------------------------------------------------------ // Hive Sync Options diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java index a76e00816189a..c6a1b0068aa50 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java @@ -253,17 +253,18 @@ void testSetupCleaningOptionsForSource() { final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2"); final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1); final Configuration conf1 = tableSource1.getConf(); - assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(20)); - assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(30)); + assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue())); + assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(FlinkOptions.ARCHIVE_MAX_COMMITS.defaultValue())); // set up new retains commits that is greater than min archive commits - this.conf.setString(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "25"); + final int retainCommits = FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue() + 5; + this.conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), retainCommits); final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2"); final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2); final Configuration conf2 = tableSource2.getConf(); - assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(35)); - assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(45)); + assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(retainCommits + 10)); + assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(retainCommits + 20)); } @Test @@ -401,17 +402,18 @@ void testSetupCleaningOptionsForSink() { final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2"); final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1); final Configuration conf1 = tableSink1.getConf(); - assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(20)); - assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(30)); + assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue())); + assertThat(conf1.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(FlinkOptions.ARCHIVE_MAX_COMMITS.defaultValue())); // set up new retains commits that is greater than min archive commits - this.conf.setString(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "25"); + final int retainCommits = FlinkOptions.ARCHIVE_MIN_COMMITS.defaultValue() + 5; + this.conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), retainCommits); final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema1, "f2"); final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2); final Configuration conf2 = tableSink2.getConf(); - assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(35)); - assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(45)); + assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), is(retainCommits + 10)); + assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(retainCommits + 20)); } @Test From 4b75cb6f23a0708fd55659cadf691d30086f5f13 Mon Sep 17 00:00:00 2001 From: peanut-chenzhong <58263343+peanut-chenzhong@users.noreply.github.com> Date: Mon, 14 Mar 2022 16:40:38 +0800 Subject: [PATCH 03/33] fix NPE when run schdule using spark-sql if the commits time < hoodie.compact.inline.max.delta.commits (#4976) * Update CompactionHoodiePathCommand.scala fix NPE when run schdule using spark-sql if the commits time < hoodie.compact.inline.max.delta.commits * Update CompactionHoodiePathCommand.scala fix IndexOutOfBoundsException when there`s no schedule for compaction * Update CompactionHoodiePathCommand.scala fix CI issue --- .../sql/hudi/command/CompactionHoodiePathCommand.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala index 1135981a92acd..7bd9a3f229b29 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala @@ -50,7 +50,7 @@ case class CompactionHoodiePathCommand(path: String, if (client.scheduleCompactionAtInstant(instantTime, HOption.empty[java.util.Map[String, String]])) { Seq(Row(instantTime)) } else { - Seq(Row(null)) + Seq.empty[Row] } case RUN => // Do compaction @@ -64,8 +64,12 @@ case class CompactionHoodiePathCommand(path: String, pendingCompactionInstants } else { // If there are no pending compaction, schedule to generate one. // CompactionHoodiePathCommand will return instanceTime for SCHEDULE. - Seq(CompactionHoodiePathCommand(path, CompactionOperation.SCHEDULE) - .run(sparkSession).take(1).get(0).getString(0)).filter(_ != null) + val scheduleSeq = CompactionHoodiePathCommand(path, CompactionOperation.SCHEDULE).run(sparkSession) + if (scheduleSeq.isEmpty) { + Seq.empty + } else { + Seq(scheduleSeq.take(1).get(0).getString(0)).filter(_ != null) + } } } else { // Check if the compaction timestamp has exists in the pending compaction From 003c6ee73e6809f90347e2e81696d2eb8ab2af2d Mon Sep 17 00:00:00 2001 From: wangxianghu Date: Mon, 14 Mar 2022 18:24:06 +0400 Subject: [PATCH 04/33] [MINODR] Remove repeated kafka-clients dependencies (#5034) --- packaging/hudi-utilities-bundle/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 1ffca7634a1ff..0685baee0a9ff 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -126,7 +126,6 @@ io.confluent:common-config io.confluent:common-utils io.confluent:kafka-schema-registry-client - org.apache.kafka:kafka-clients io.dropwizard.metrics:metrics-core io.dropwizard.metrics:metrics-graphite io.dropwizard.metrics:metrics-jmx From 22c3ce73dbaca70ec0eceee721bb9fbd344c2331 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 14 Mar 2022 15:34:17 -0700 Subject: [PATCH 05/33] [HUDI-3621] Fixing NullPointerException in DeltaStreamer (#5039) --- .../org/apache/hudi/utilities/deltastreamer/DeltaSync.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 2a319308fbfea..7dd9a36892cf2 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 @@ -910,6 +910,10 @@ public Option getCommitTimelineOpt() { * @return Requested clustering instant. */ public Option getClusteringInstantOpt() { - return writeClient.scheduleClustering(Option.empty()); + if (writeClient != null) { + return writeClient.scheduleClustering(Option.empty()); + } else { + return Option.empty(); + } } } From 30cf39301e311e26b29a65b3e4a229da92f925d1 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 14 Mar 2022 15:39:26 -0700 Subject: [PATCH 06/33] [HUDI-3623] Removing hive sync node from non hive yamls (#5040) --- .../test-suite/cow-spark-long-running.yaml | 16 ++-------------- .../config/test-suite/cow-spark-simple.yaml | 16 ++-------------- ...long-running-multi-partitions-metadata.yaml | 8 +------- ...streamer-long-running-multi-partitions.yaml | 8 +------- ...treamer-medium-full-dataset-validation.yaml | 8 +------- .../detlastreamer-long-running-example.yaml | 18 ++---------------- .../test-suite/insert-overwrite-table.yaml | 16 ++-------------- .../config/test-suite/insert-overwrite.yaml | 16 ++-------------- .../config/test-suite/simple-clustering.yaml | 16 ++-------------- .../test-suite/simple-deltastreamer.yaml | 16 ++-------------- .../config/test-suite/spark-clustering.yaml | 8 +------- 11 files changed, 18 insertions(+), 128 deletions(-) diff --git a/docker/demo/config/test-suite/cow-spark-long-running.yaml b/docker/demo/config/test-suite/cow-spark-long-running.yaml index 8229ba749502e..c25b95c8da457 100644 --- a/docker/demo/config/test-suite/cow-spark-long-running.yaml +++ b/docker/demo/config/test-suite/cow-spark-long-running.yaml @@ -25,17 +25,11 @@ dag_content: num_records_insert: 10000 type: SparkInsertNode deps: none - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_insert first_validate: config: validate_hive: false type: ValidateDatasetNode - deps: first_hive_sync + deps: first_insert first_upsert: config: record_size: 200 @@ -52,18 +46,12 @@ dag_content: num_records_delete: 8000 type: SparkDeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: true type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete last_validate: config: execute_itr_count: 30 diff --git a/docker/demo/config/test-suite/cow-spark-simple.yaml b/docker/demo/config/test-suite/cow-spark-simple.yaml index 0859c63200203..192adcf377dc0 100644 --- a/docker/demo/config/test-suite/cow-spark-simple.yaml +++ b/docker/demo/config/test-suite/cow-spark-simple.yaml @@ -25,17 +25,11 @@ dag_content: num_records_insert: 100 type: SparkInsertNode deps: none - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_insert first_validate: config: validate_hive: false type: ValidateDatasetNode - deps: first_hive_sync + deps: first_insert first_upsert: config: record_size: 1000 @@ -52,15 +46,9 @@ dag_content: num_records_delete: 30 type: SparkDeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: second_hive_sync \ No newline at end of file + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml index 817552224e8f3..0212fdf43c512 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml @@ -57,18 +57,12 @@ dag_content: num_records_delete: 8000 type: DeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: true type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete last_validate: config: execute_itr_count: 30 diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml index ba490a8c194dd..d7b11194736e9 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml @@ -57,18 +57,12 @@ dag_content: num_records_delete: 8000 type: DeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: true type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete last_validate: config: execute_itr_count: 50 diff --git a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml index aa83b2780127c..77898640ea144 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml @@ -60,18 +60,12 @@ dag_content: num_records_delete: 8000 type: DeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete last_validate: config: execute_itr_count: 20 diff --git a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml index 28578eb9b687e..4b2ee7ad13cc4 100644 --- a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml +++ b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml @@ -41,17 +41,11 @@ dag_content: num_records_insert: 300 deps: second_insert type: InsertNode - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: third_insert first_validate: config: validate_hive: false type: ValidateDatasetNode - deps: first_hive_sync + deps: third_insert first_upsert: config: record_size: 1000 @@ -68,22 +62,14 @@ dag_content: num_records_delete: 8000 type: DeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: true type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete last_validate: config: execute_itr_count: 50 - validate_clean: true - validate_archival: true type: ValidateAsyncOperations deps: second_validate diff --git a/docker/demo/config/test-suite/insert-overwrite-table.yaml b/docker/demo/config/test-suite/insert-overwrite-table.yaml index 8b5a26e4683b7..1a58abdcc4789 100644 --- a/docker/demo/config/test-suite/insert-overwrite-table.yaml +++ b/docker/demo/config/test-suite/insert-overwrite-table.yaml @@ -53,19 +53,13 @@ dag_content: num_partitions_upsert: 10 type: SparkUpsertNode deps: second_insert - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: second_upsert first_insert_overwrite_table: config: record_size: 1000 repeat_count: 10 num_records_insert: 10 type: SparkInsertOverwriteTableNode - deps: first_hive_sync + deps: second_upsert delete_all_input_except_last: config: delete_input_data_except_latest: true @@ -89,16 +83,10 @@ dag_content: num_partitions_upsert: 10 type: SparkUpsertNode deps: third_insert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: third_upsert second_validate: config: validate_full_data : true validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: second_hive_sync + deps: third_upsert diff --git a/docker/demo/config/test-suite/insert-overwrite.yaml b/docker/demo/config/test-suite/insert-overwrite.yaml index f2299c50c08f3..dc185d5938f6d 100644 --- a/docker/demo/config/test-suite/insert-overwrite.yaml +++ b/docker/demo/config/test-suite/insert-overwrite.yaml @@ -54,12 +54,6 @@ dag_content: num_partitions_upsert: 10 type: SparkUpsertNode deps: second_insert - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: second_upsert first_insert_overwrite: config: record_size: 1000 @@ -67,7 +61,7 @@ dag_content: repeat_count: 1 num_records_insert: 10 type: SparkInsertOverwriteNode - deps: first_hive_sync + deps: second_upsert delete_all_input_except_last: config: delete_input_data_except_latest: true @@ -91,16 +85,10 @@ dag_content: num_partitions_upsert: 10 type: SparkUpsertNode deps: third_insert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: third_upsert second_validate: config: validate_full_data : true validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: second_hive_sync + deps: third_upsert diff --git a/docker/demo/config/test-suite/simple-clustering.yaml b/docker/demo/config/test-suite/simple-clustering.yaml index 7389ee3ebc34b..01849bb6436ea 100644 --- a/docker/demo/config/test-suite/simple-clustering.yaml +++ b/docker/demo/config/test-suite/simple-clustering.yaml @@ -47,30 +47,18 @@ dag_content: num_records_delete: 9000 type: DeleteNode deps: third_insert - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete first_validate: config: validate_hive: false type: ValidateDatasetNode - deps: first_hive_sync + deps: first_delete first_cluster: config: execute_itr_count: 25 type: ClusteringNode deps: first_validate - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_cluster second_validate: config: validate_hive: false type: ValidateDatasetNode - deps: second_hive_sync + deps: first_cluster diff --git a/docker/demo/config/test-suite/simple-deltastreamer.yaml b/docker/demo/config/test-suite/simple-deltastreamer.yaml index f49a41baf8541..11b7f17d34a4d 100644 --- a/docker/demo/config/test-suite/simple-deltastreamer.yaml +++ b/docker/demo/config/test-suite/simple-deltastreamer.yaml @@ -41,17 +41,11 @@ dag_content: num_records_insert: 300 deps: second_insert type: InsertNode - first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: third_insert first_validate: config: validate_hive: false type: ValidateDatasetNode - deps: first_hive_sync + deps: third_insert first_upsert: config: record_size: 1000 @@ -68,15 +62,9 @@ dag_content: num_records_delete: 2000 type: DeleteNode deps: first_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: true type: ValidateDatasetNode - deps: second_hive_sync + deps: first_delete diff --git a/docker/demo/config/test-suite/spark-clustering.yaml b/docker/demo/config/test-suite/spark-clustering.yaml index e8e722ca77c7c..8da4f953983b8 100644 --- a/docker/demo/config/test-suite/spark-clustering.yaml +++ b/docker/demo/config/test-suite/spark-clustering.yaml @@ -59,15 +59,9 @@ dag_content: num_records_delete: 16000 type: SparkDeleteNode deps: second_upsert - second_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete second_validate: config: validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: second_hive_sync \ No newline at end of file + deps: first_delete \ No newline at end of file From d40adfa2d717da683787afbc6bb58e409b9430c5 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Mon, 14 Mar 2022 16:14:00 -0700 Subject: [PATCH 07/33] [HUDI-3620] Adding spark3.2.0 profile (#5038) --- pom.xml | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/pom.xml b/pom.xml index 33bd112a1bfe3..5be49eb04c180 100644 --- a/pom.xml +++ b/pom.xml @@ -1636,6 +1636,36 @@ + + spark3.2.0 + + 3.2.0 + ${spark3.version} + ${spark3.version} + ${scala12.version} + 2.12 + hudi-spark3 + hudi-spark3-common + 3.1.0 + 2.4.1 + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + true + true + + + hudi-spark-datasource/hudi-spark3 + hudi-spark-datasource/hudi-spark3-common + + + + spark3.2.0 + + + + spark3.1.x From 3b59b769528e0953bf921bd96ffb68e12c3524d4 Mon Sep 17 00:00:00 2001 From: wangxianghu Date: Tue, 15 Mar 2022 15:06:30 +0400 Subject: [PATCH 08/33] [HUDI-3547] Introduce MaxwellSourcePostProcessor to extract data from Maxwell json string (#4987) * [HUDI-3547] Introduce MaxwellSourcePostProcessor to extract data from Maxwell json string * add ut * Address comment --- .../hudi/common/util/DateTimeUtils.java | 17 ++ .../MaxwellJsonKafkaSourcePostProcessor.java | 190 ++++++++++++++++++ .../maxwell/PreCombineFieldType.java | 44 ++++ .../TestJsonKafkaSourcePostProcessor.java | 130 ++++++++++++ 4 files changed, 381 insertions(+) create mode 100644 hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java create mode 100644 hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java index e52e5660957dd..531a0903f66fc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/DateTimeUtils.java @@ -21,6 +21,9 @@ import java.time.Duration; import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; import java.time.temporal.ChronoUnit; import java.util.Arrays; @@ -126,6 +129,20 @@ private static Map initMap() { return labelToUnit; } + /** + * Convert UNIX_TIMESTAMP to string in given format. + * + * @param unixTimestamp UNIX_TIMESTAMP + * @param timeFormat string time format + */ + public static String formatUnixTimestamp(long unixTimestamp, String timeFormat) { + ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(timeFormat)); + DateTimeFormatter dtf = DateTimeFormatter.ofPattern(timeFormat); + return LocalDateTime + .ofInstant(Instant.ofEpochSecond(unixTimestamp), ZoneId.systemDefault()) + .format(dtf); + } + /** * Enum which defines time unit, mostly used to parse value from configuration file. */ diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java new file mode 100644 index 0000000000000..9ca91893bec69 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/MaxwellJsonKafkaSourcePostProcessor.java @@ -0,0 +1,190 @@ +/* + * 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.processor.maxwell; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.DateTimeUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; +import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Locale; +import java.util.Objects; +import java.util.regex.Pattern; + +import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.DATE_STRING; +import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.EPOCHMILLISECONDS; +import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.NON_TIMESTAMP; +import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.UNIX_TIMESTAMP; +import static org.apache.hudi.utilities.sources.processor.maxwell.PreCombineFieldType.valueOf; + +/** + * A {@link JsonKafkaSourcePostProcessor} help to extract fresh data from maxwell json string and tag the record as + * delete or not. + */ +public class MaxwellJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor { + + private static final Logger LOG = LogManager.getLogger(MaxwellJsonKafkaSourcePostProcessor.class); + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + public MaxwellJsonKafkaSourcePostProcessor(TypedProperties props) { + super(props); + } + + // ------------------------------------------------------------------------ + // Partial fields in maxwell json string + // ------------------------------------------------------------------------ + + private static final String DATABASE = "database"; + private static final String TABLE = "table"; + private static final String DATA = "data"; + private static final String OPERATION_TYPE = "type"; + private static final String TS = "ts"; + + // ------------------------------------------------------------------------ + // Operation types + // ------------------------------------------------------------------------ + + private static final String INSERT = "insert"; + private static final String UPDATE = "update"; + private static final String DELETE = "delete"; + + /** + * Configs to be passed for this processor. + */ + public static class Config { + public static final ConfigProperty DATABASE_NAME_REGEX_PROP = ConfigProperty + .key("hoodie.deltastreamer.source.json.kafka.post.processor.maxwell.database.regex") + .noDefaultValue() + .withDocumentation("Database name regex."); + + public static final ConfigProperty TABLE_NAME_REGEX_PROP = ConfigProperty + .key("hoodie.deltastreamer.source.json.kafka.post.processor.maxwell.table.regex") + .noDefaultValue() + .withDocumentation("Table name regex."); + + public static final ConfigProperty PRECOMBINE_FIELD_TYPE_PROP = ConfigProperty + .key("hoodie.deltastreamer.source.json.kafka.post.processor.maxwell.precombine.field.type") + .defaultValue("DATA_STRING") + .withDocumentation("Data type of the preCombine field. could be NON_TIMESTAMP, DATE_STRING," + + "UNIX_TIMESTAMP or EPOCHMILLISECONDS. DATA_STRING by default "); + + public static final ConfigProperty PRECOMBINE_FIELD_FORMAT_PROP = ConfigProperty + .key("hoodie.deltastreamer.source.json.kafka.post.processor.maxwell.precombine.field.format") + .defaultValue("yyyy-MM-dd HH:mm:ss") + .withDocumentation("When the preCombine filed is in DATE_STRING format, use should tell hoodie" + + "what format it is. 'yyyy-MM-dd HH:mm:ss' by default"); + } + + @Override + public JavaRDD process(JavaRDD maxwellJsonRecords) { + return maxwellJsonRecords.map(record -> { + JsonNode inputJson = MAPPER.readTree(record); + String database = inputJson.get(DATABASE).textValue(); + String table = inputJson.get(TABLE).textValue(); + + // filter out target databases and tables + if (isTargetTable(database, table)) { + + LOG.info(String.format("Maxwell source processor starts process table : %s.%s", database, table)); + + ObjectNode result = (ObjectNode) inputJson.get(DATA); + String type = inputJson.get(OPERATION_TYPE).textValue(); + + // insert or update + if (INSERT.equals(type) || UPDATE.equals(type)) { + // tag this record not delete. + result.put(HoodieRecord.HOODIE_IS_DELETED, false); + return result.toString(); + + // delete + } else if (DELETE.equals(type)) { + return processDelete(inputJson, result); + } else { + // there might be some ddl data, ignore it + return null; + } + } else { + // not the data from target table(s), ignore it + return null; + } + }).filter(Objects::nonNull); + } + + private String processDelete(JsonNode inputJson, ObjectNode result) { + // tag this record as delete. + result.put(HoodieRecord.HOODIE_IS_DELETED, true); + + PreCombineFieldType preCombineFieldType = + valueOf(this.props.getString(Config.PRECOMBINE_FIELD_TYPE_PROP.key(), + Config.PRECOMBINE_FIELD_TYPE_PROP.defaultValue()).toUpperCase(Locale.ROOT)); + + // maxwell won't update the `update_time`(delete time) field of the record which is tagged as delete. so if we + // want to delete this record correctly, we should update its `update_time` to a time closer to where the + // delete operation actually occurred. here we use `ts` from maxwell json string as this 'delete' time. + + // we can update the `update_time`(delete time) only when it is in timestamp format. + if (!preCombineFieldType.equals(NON_TIMESTAMP)) { + String preCombineField = this.props.getString(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), + HoodieWriteConfig.PRECOMBINE_FIELD_NAME.defaultValue()); + + // ts from maxwell + long ts = inputJson.get(TS).longValue(); + + // convert the `update_time`(delete time) to the proper format. + if (preCombineFieldType.equals(DATE_STRING)) { + // DATE_STRING format + String timeFormat = this.props.getString(Config.PRECOMBINE_FIELD_FORMAT_PROP.key(), Config.PRECOMBINE_FIELD_FORMAT_PROP.defaultValue()); + result.put(preCombineField, DateTimeUtils.formatUnixTimestamp(ts, timeFormat)); + } else if (preCombineFieldType.equals(EPOCHMILLISECONDS)) { + // EPOCHMILLISECONDS format + result.put(preCombineField, ts * 1000L); + } else if (preCombineFieldType.equals(UNIX_TIMESTAMP)) { + // UNIX_TIMESTAMP format + result.put(preCombineField, ts); + } else { + throw new HoodieSourcePostProcessException("Unsupported preCombine time format " + preCombineFieldType); + } + } + return result.toString(); + } + + /** + * Check if it is the right table we want to consume from. + * + * @param database database the data belong to + * @param table table the data belong to + */ + private boolean isTargetTable(String database, String table) { + String databaseRegex = this.props.getString(Config.DATABASE_NAME_REGEX_PROP.key()); + String tableRegex = this.props.getString(Config.TABLE_NAME_REGEX_PROP.key()); + return Pattern.matches(databaseRegex, database) && Pattern.matches(tableRegex, table); + } + +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java new file mode 100644 index 0000000000000..d3969a02bc4f3 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/maxwell/PreCombineFieldType.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.processor.maxwell; + +/** + * Enum of preCombine field time type. + */ +public enum PreCombineFieldType { + /** + * Not a timestamp type field + */ + NON_TIMESTAMP, + + /** + * Timestamp type field in string format. + */ + DATE_STRING, + + /** + * Timestamp type field in UNIX_TIMESTAMP format. + */ + UNIX_TIMESTAMP, + + /** + * Timestamp type field in EPOCHMILLISECONDS format. + */ + EPOCHMILLISECONDS +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java index bd150ed29be38..80ac2f921ecd5 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java @@ -18,24 +18,36 @@ package org.apache.hudi.utilities.sources; +import com.fasterxml.jackson.databind.JsonNode; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.DateTimeUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; +import org.apache.hudi.utilities.sources.processor.maxwell.MaxwellJsonKafkaSourcePostProcessor; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; import java.util.Objects; import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.JSON_KAFKA_PROCESSOR_CLASS_OPT; import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.jsonifyRecords; 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.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestJsonKafkaSourcePostProcessor extends TestJsonKafkaSource { @@ -120,6 +132,124 @@ public void testChainedJsonKafkaSourcePostProcessor() { assertEquals(0, fetch1.getBatch().get().count()); } + @Test + public void testMaxwellJsonKafkaSourcePostProcessor() throws IOException { + // ------------------------------------------------------------------------ + // Maxwell data + // ------------------------------------------------------------------------ + + // database hudi, table hudi_maxwell_01 (insert, update and delete) + String hudiMaxwell01Insert = "{\"database\":\"hudi\",\"table\":\"hudi_maxwell_01\",\"type\":\"insert\"," + + "\"ts\":1647074402,\"xid\":6233,\"commit\":true,\"data\":{\"id\":\"6018220e39e74477b45c7cf42f66bdc0\"," + + "\"name\":\"mathieu\",\"age\":18,\"insert_time\":\"2022-03-12 08:40:02\"," + + "\"update_time\":\"2022-03-12 08:40:02\"}}"; + + String hudiMaxwell01Update = "{\"database\":\"hudi\",\"table\":\"hudi_maxwell_01\",\"type\":\"update\"," + + "\"ts\":1647074482,\"xid\":6440,\"commit\":true,\"data\":{\"id\":\"6018220e39e74477b45c7cf42f66bdc0\"," + + "\"name\":\"mathieu\",\"age\":20,\"insert_time\":\"2022-03-12 04:40:02\",\"update_time\":\"2022-03-12 04:42:25\"}," + + "\"old\":{\"age\":18,\"insert_time\":\"2022-03-12 08:40:02\",\"update_time\":\"2022-03-12 08:40:02\"}}"; + + String hudiMaxwell01Delete = "{\"database\":\"hudi\",\"table\":\"hudi_maxwell_01\",\"type\":\"delete\"," + + "\"ts\":1647074555,\"xid\":6631,\"commit\":true,\"data\":{\"id\":\"6018220e39e74477b45c7cf42f66bdc0\"," + + "\"name\":\"mathieu\",\"age\":20,\"insert_time\":\"2022-03-12 04:40:02\",\"update_time\":\"2022-03-12 04:42:25\"}}"; + + String hudiMaxwell01Ddl = "{\"type\":\"table-alter\",\"database\":\"hudi\",\"table\":\"hudi_maxwell_01\"," + + "\"old\":{\"database\":\"hudi\",\"charset\":\"utf8\",\"table\":\"hudi_maxwell_01\"," + + "\"primary-key\":[\"id\"],\"columns\":[{\"type\":\"varchar\",\"name\":\"id\",\"charset\":\"utf8\"}," + + "{\"type\":\"varchar\",\"name\":\"name\",\"charset\":\"utf8\"},{\"type\":\"int\",\"name\":\"age\"," + + "\"signed\":true},{\"type\":\"timestamp\",\"name\":\"insert_time\",\"column-length\":0}," + + "{\"type\":\"timestamp\",\"name\":\"update_time\",\"column-length\":0}]},\"def\":{\"database\":\"hudi\"," + + "\"charset\":\"utf8\",\"table\":\"hudi_maxwell_01\",\"primary-key\":[\"id\"]," + + "\"columns\":[{\"type\":\"varchar\",\"name\":\"id\",\"charset\":\"utf8\"},{\"type\":\"varchar\"," + + "\"name\":\"name\",\"charset\":\"utf8\"},{\"type\":\"int\",\"name\":\"age\",\"signed\":true}," + + "{\"type\":\"timestamp\",\"name\":\"insert_time\",\"column-length\":0},{\"type\":\"timestamp\"," + + "\"name\":\"update_time\",\"column-length\":0}]},\"ts\":1647072305000,\"sql\":\"/* ApplicationName=DBeaver " + + "21.0.4 - Main */ ALTER TABLE hudi.hudi_maxwell_01 MODIFY COLUMN age int(3) NULL\"}"; + + // database hudi, table hudi_maxwell_010, insert + String hudiMaxwell010Insert = "{\"database\":\"hudi\",\"table\":\"hudi_maxwell_010\",\"type\":\"insert\"," + + "\"ts\":1647073982,\"xid\":5164,\"commit\":true,\"data\":{\"id\":\"f3eaf4cdf7534e47a88cdf93d19b2ee6\"," + + "\"name\":\"wangxianghu\",\"age\":18,\"insert_time\":\"2022-03-12 08:33:02\"," + + "\"update_time\":\"2022-03-12 08:33:02\"}}"; + + // database hudi_02, table hudi_maxwell_02, insert + String hudi02Maxwell02Insert = "{\"database\":\"hudi_02\",\"table\":\"hudi_maxwell_02\",\"type\":\"insert\"," + + "\"ts\":1647073916,\"xid\":4990,\"commit\":true,\"data\":{\"id\":\"9bb17f316ee8488cb107621ddf0f3cb0\"," + + "\"name\":\"andy\",\"age\":17,\"insert_time\":\"2022-03-12 08:31:56\"," + + "\"update_time\":\"2022-03-12 08:31:56\"}}"; + + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + ObjectMapper mapper = new ObjectMapper(); + TypedProperties props = new TypedProperties(); + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.DATABASE_NAME_REGEX_PROP.key(), "hudi(_)?[0-9]{0,2}"); + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.TABLE_NAME_REGEX_PROP.key(), "hudi_maxwell(_)?[0-9]{0,2}"); + + // test insert and update + JavaRDD inputInsertAndUpdate = jsc().parallelize(Arrays.asList(hudiMaxwell01Insert, hudiMaxwell01Update)); + MaxwellJsonKafkaSourcePostProcessor processor = new MaxwellJsonKafkaSourcePostProcessor(props); + processor.process(inputInsertAndUpdate).map(mapper::readTree).foreach(record -> { + // database name should be null + JsonNode database = record.get("database"); + // insert and update records should be tagged as no delete + boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED).booleanValue(); + + assertFalse(isDelete); + assertNull(database); + }); + + // test delete + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.PRECOMBINE_FIELD_TYPE_PROP.key(), "DATE_STRING"); + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.PRECOMBINE_FIELD_FORMAT_PROP.key(), "yyyy-MM-dd HH:mm:ss"); + props.setProperty(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "update_time"); + + JavaRDD inputDelete = jsc().parallelize(Collections.singletonList(hudiMaxwell01Delete)); + + long ts = mapper.readTree(hudiMaxwell01Delete).get("ts").longValue(); + String formatTs = DateTimeUtils.formatUnixTimestamp(ts, "yyyy-MM-dd HH:mm:ss"); + + new MaxwellJsonKafkaSourcePostProcessor(props) + .process(inputDelete).map(mapper::readTree).foreach(record -> { + + // delete records should be tagged as delete + boolean isDelete = record.get(HoodieRecord.HOODIE_IS_DELETED).booleanValue(); + // update_time should equals ts + String updateTime = record.get("update_time").textValue(); + + assertEquals(formatTs, updateTime); + assertTrue(isDelete); + }); + + // test preCombine field is not time + props.setProperty(MaxwellJsonKafkaSourcePostProcessor.Config.PRECOMBINE_FIELD_TYPE_PROP.key(), "NON_TIMESTAMP"); + props.setProperty(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "id"); + + JavaRDD inputDelete2 = jsc().parallelize(Collections.singletonList(hudiMaxwell01Delete)); + + String updateTimeInUpdate = mapper.readTree(hudiMaxwell01Update).get("data").get("update_time").textValue(); + new MaxwellJsonKafkaSourcePostProcessor(props) + .process(inputDelete2).map(mapper::readTree).foreach(record -> { + + // updateTimeInUpdate should updateTimeInDelete + String updateTimeInDelete = record.get("update_time").textValue(); + assertEquals(updateTimeInUpdate, updateTimeInDelete); + }); + + // test database, table regex + JavaRDD dirtyData = jsc().parallelize(Arrays.asList(hudiMaxwell01Insert, hudiMaxwell010Insert, hudi02Maxwell02Insert)); + long validDataNum = processor.process(dirtyData).count(); + // hudiMaxwell010Insert is dirty data + assertEquals(2, validDataNum); + + // test ddl + JavaRDD ddlData = jsc().parallelize(Collections.singletonList(hudiMaxwell01Ddl)); + // ddl data will be ignored, ths count should be 0 + long ddlDataNum = processor.process(ddlData).count(); + assertEquals(0, ddlDataNum); + } + /** * JsonKafkaSourcePostProcessor that return a sub RDD of the incoming data which get the data from incoming data using * {org.apache.spark.api.java.JavaRDD#sample(boolean, double, long)} method. From 6ed7106e5904ce5bf4741b368245101b9ec5836d Mon Sep 17 00:00:00 2001 From: Thinking Chen Date: Tue, 15 Mar 2022 19:06:50 +0800 Subject: [PATCH 09/33] [HUDI-3606] Add `org.objenesis:objenesis` to hudi-timeline-server-bundle pom (#5017) --- packaging/hudi-timeline-server-bundle/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 618d3d2122315..18f7c9665a20f 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -206,6 +206,7 @@ com.esotericsoftware:kryo-shaded com.esotericsoftware:minlog log4j:log4j + org.objenesis:objenesis From 9bdda2a312bd830bc1659a5305fc28b4469d07be Mon Sep 17 00:00:00 2001 From: l-shen Date: Tue, 15 Mar 2022 20:34:31 +0800 Subject: [PATCH 10/33] [HUDI-3619] Fix HoodieOperation fromValue using wrong constant value (#5033) Co-authored-by: root --- .../main/java/org/apache/hudi/common/model/HoodieOperation.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieOperation.java index 0da40eb0a64bf..47625820a7fbe 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieOperation.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieOperation.java @@ -40,7 +40,7 @@ public enum HoodieOperation { /** * Delete operation. */ - DELETE("D", (byte) 4); + DELETE("D", (byte) 3); private final String name; private final byte value; From 5e8ff8d793f5c50edef2865bb8733301a43f495c Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Tue, 15 Mar 2022 10:38:36 -0700 Subject: [PATCH 11/33] [HUDI-3514] Rebase Data Skipping flow to rely on MT Column Stats index (#4948) --- .../columnstats/ColumnStatsIndexHelper.java | 23 +- .../testutils/HoodieMergeOnReadTestUtils.java | 21 +- hudi-common/src/main/avro/HoodieMetadata.avsc | 8 + .../common/table/HoodieTableMetaClient.java | 1 + .../common/table/TableSchemaResolver.java | 9 +- .../hudi/common/util/TablePathUtils.java | 33 +-- .../hudi/metadata/HoodieMetadataPayload.java | 19 +- .../hudi/metadata/HoodieTableMetadata.java | 9 +- .../metadata/HoodieTableMetadataUtil.java | 11 +- .../hudi/common/util/TestTablePathUtils.java | 20 +- .../HoodieCopyOnWriteTableInputFormat.java | 13 +- .../scala/org/apache/hudi/DefaultSource.scala | 10 +- .../org/apache/hudi/HoodieDatasetUtils.scala | 45 ++++ .../org/apache/hudi/HoodieFileIndex.scala | 204 ++++++++++-------- .../hudi/SparkHoodieTableFileIndex.scala | 2 +- .../spark/sql/hudi/DataSkippingUtils.scala | 71 ++---- .../apache/hudi/TestDataSkippingUtils.scala | 4 +- .../org/apache/hudi/TestHoodieFileIndex.scala | 75 +++++-- .../functional/TestLayoutOptimization.scala | 3 + 19 files changed, 358 insertions(+), 223 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetUtils.scala diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java index 4fdb6a6be3cba..b98893344b8ec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java @@ -80,22 +80,21 @@ public class ColumnStatsIndexHelper { private static final String SPARK_JOB_DESCRIPTION = "spark.job.description"; - private static final String Z_INDEX_FILE_COLUMN_NAME = "file"; - - private static final String Z_INDEX_MIN_VALUE_STAT_NAME = "minValue"; - private static final String Z_INDEX_MAX_VALUE_STAT_NAME = "maxValue"; - private static final String Z_INDEX_NUM_NULLS_STAT_NAME = "num_nulls"; + private static final String COLUMN_STATS_INDEX_FILE_COLUMN_NAME = "file"; + private static final String COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME = "minValue"; + private static final String COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME = "maxValue"; + private static final String COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME = "num_nulls"; public static String getMinColumnNameFor(String colName) { - return composeZIndexColName(colName, Z_INDEX_MIN_VALUE_STAT_NAME); + return composeZIndexColName(colName, COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME); } public static String getMaxColumnNameFor(String colName) { - return composeZIndexColName(colName, Z_INDEX_MAX_VALUE_STAT_NAME); + return composeZIndexColName(colName, COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME); } public static String getNumNullsColumnNameFor(String colName) { - return composeZIndexColName(colName, Z_INDEX_NUM_NULLS_STAT_NAME); + return composeZIndexColName(colName, COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME); } /** @@ -407,11 +406,11 @@ private static Dataset tryMergeMostRecentIndexTableInto( @Nonnull public static StructType composeIndexSchema(@Nonnull List zorderedColumnsSchemas) { List schema = new ArrayList<>(); - schema.add(new StructField(Z_INDEX_FILE_COLUMN_NAME, StringType$.MODULE$, true, Metadata.empty())); + schema.add(new StructField(COLUMN_STATS_INDEX_FILE_COLUMN_NAME, StringType$.MODULE$, true, Metadata.empty())); zorderedColumnsSchemas.forEach(colSchema -> { - schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MIN_VALUE_STAT_NAME, colSchema.dataType())); - schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MAX_VALUE_STAT_NAME, colSchema.dataType())); - schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_NUM_NULLS_STAT_NAME, LongType$.MODULE$)); + schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_MIN_VALUE_STAT_NAME, colSchema.dataType())); + schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_MAX_VALUE_STAT_NAME, colSchema.dataType())); + schema.add(composeColumnStatStructType(colSchema.name(), COLUMN_STATS_INDEX_NUM_NULLS_STAT_NAME, LongType$.MODULE$)); }); return StructType$.MODULE$.apply(schema); } 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 8cd68ff8335d8..931714fd2fcf6 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 @@ -62,10 +62,10 @@ public static List getRecordsUsingInputFormat(Configuration conf, } public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, - String basePath, JobConf jobConf, boolean realtime, boolean populateMetaFieldsConfigValue) { + String basePath, JobConf jobConf, boolean realtime, boolean populateMetaFields) { Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA); return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema, - HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>(), populateMetaFieldsConfigValue); + HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>(), populateMetaFields); } public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema, @@ -74,14 +74,23 @@ public static List getRecordsUsingInputFormat(Configuration conf, } public static List getRecordsUsingInputFormat(Configuration conf, List inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema, - String rawHiveColumnTypes, boolean projectCols, List projectedColumns, boolean populateMetaFieldsConfigValue) { + String rawHiveColumnTypes, boolean projectCols, List projectedColumns, boolean populateMetaFields) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(conf).setBasePath(basePath).build(); FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), realtime, jobConf); - Schema schema = HoodieAvroUtils.addMetadataFields(rawSchema); - String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes); - setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns, populateMetaFieldsConfigValue); + Schema schema; + String hiveColumnTypes; + + if (populateMetaFields) { + schema = HoodieAvroUtils.addMetadataFields(rawSchema); + hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes); + } else { + schema = rawSchema; + hiveColumnTypes = rawHiveColumnTypes; + } + + setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns, populateMetaFields); final List fields; if (projectCols) { fields = schema.getFields().stream().filter(f -> projectedColumns.contains(f.name())) diff --git a/hudi-common/src/main/avro/HoodieMetadata.avsc b/hudi-common/src/main/avro/HoodieMetadata.avsc index 4037dd0f1ab01..a6807d58e48e0 100644 --- a/hudi-common/src/main/avro/HoodieMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieMetadata.avsc @@ -109,6 +109,14 @@ "string" ] }, + { + "doc": "Column name for which this column statistics applies", + "name": "columnName", + "type": [ + "null", + "string" + ] + }, { "doc": "Minimum value in the range. Based on user data table schema, we can convert this to appropriate type", "name": "minValue", diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 4c1eac79dc413..3de93005da66d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -83,6 +83,7 @@ public class HoodieTableMetaClient implements Serializable { public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".aux"; public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + Path.SEPARATOR + ".bootstrap"; public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".heartbeat"; + public static final String METADATA_TABLE_FOLDER_PATH = METAFOLDER_NAME + Path.SEPARATOR + "metadata"; public static final String COLUMN_STATISTICS_INDEX_NAME = ".colstatsindex"; public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR + ".partitions"; 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 f2db4d692866a..a0771d1249c75 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 @@ -58,6 +58,8 @@ /** * Helper class to read schema from data files and log files and to convert it between different formats. + * + * TODO(HUDI-3626) cleanup */ public class TableSchemaResolver { @@ -143,7 +145,7 @@ public Schema getTableAvroSchemaFromDataFile() { * @throws Exception */ public Schema getTableAvroSchema() throws Exception { - return getTableAvroSchema(true); + return getTableAvroSchema(metaClient.getTableConfig().populateMetaFields()); } /** @@ -197,7 +199,10 @@ public MessageType getTableParquetSchema() throws Exception { * * @return Avro user data schema * @throws Exception + * + * @deprecated use {@link #getTableAvroSchema(boolean)} instead */ + @Deprecated public Schema getTableAvroSchemaWithoutMetadataFields() throws Exception { return getTableAvroSchema(false); } @@ -208,7 +213,9 @@ public Schema getTableAvroSchemaWithoutMetadataFields() throws Exception { * @param instant will get the instant data schema * @return Avro user data schema * @throws Exception + * @deprecated use {@link #getTableSchemaFromCommitMetadata} instead */ + @Deprecated public Schema getTableAvroSchemaWithoutMetadataFields(HoodieInstant instant) throws Exception { Option schemaFromCommitMetadata = getTableSchemaFromCommitMetadata(instant, false); if (schemaFromCommitMetadata.isPresent()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java index d17cd4bc7bd76..9d279d5328ccc 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/TablePathUtils.java @@ -50,13 +50,13 @@ public static Option getTablePath(FileSystem fs, Path path) throws HoodieE FileStatus fileStatus = fs.getFileStatus(path); Path directory = fileStatus.isFile() ? fileStatus.getPath().getParent() : fileStatus.getPath(); - if (TablePathUtils.hasTableMetadataFolder(fs, directory)) { + if (hasTableMetadataFolder(fs, directory)) { // Handle table folder itself return Option.of(directory); } // Handle metadata folder or metadata sub folder path - Option tablePath = getTablePathFromTableMetadataPath(directory); + Option tablePath = getTablePathFromMetaFolderPath(directory); if (tablePath.isPresent()) { return tablePath; } @@ -65,20 +65,20 @@ public static Option getTablePath(FileSystem fs, Path path) throws HoodieE return getTablePathFromPartitionPath(fs, directory); } - private static boolean isTableMetadataFolder(String path) { - return path != null && path.endsWith("/" + HoodieTableMetaClient.METAFOLDER_NAME); + private static boolean isInsideTableMetaFolder(String path) { + return path != null && path.contains("/" + HoodieTableMetaClient.METAFOLDER_NAME); } - private static boolean isInsideTableMetadataFolder(String path) { - return path != null && path.contains("/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"); + private static boolean isInsideMetadataTableInMetaFolder(String path) { + return path != null && path.contains("/" + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); } - private static Option getTablePathFromTableMetadataPath(Path path) { + private static Option getTablePathFromMetaFolderPath(Path path) { String pathStr = path.toString(); - if (isTableMetadataFolder(pathStr)) { - return Option.of(path.getParent()); - } else if (isInsideTableMetadataFolder(pathStr)) { + // NOTE: Since Metadata Table itself resides w/in the Meta-folder, we need to make sure + // that we don't misinterpret attempt to read MT table itself + if (isInsideTableMetaFolder(pathStr) && !isInsideMetadataTableInMetaFolder(pathStr)) { int index = pathStr.indexOf("/" + HoodieTableMetaClient.METAFOLDER_NAME); return Option.of(new Path(pathStr.substring(0, index))); } @@ -92,12 +92,21 @@ private static Option getTablePathFromPartitionPath(FileSystem fs, Path pa HoodiePartitionMetadata metadata = new HoodiePartitionMetadata(fs, partitionPath); metadata.readFromFS(); return Option.of(getNthParent(partitionPath, metadata.getPartitionDepth())); + } else { + // Simply traverse directory structure until found .hoodie folder + Path current = partitionPath; + while (current != null) { + if (hasTableMetadataFolder(fs, current)) { + return Option.of(current); + } + current = current.getParent(); + } + + return Option.empty(); } } catch (IOException ioe) { throw new HoodieException("Error reading partition metadata for " + partitionPath, ioe); } - - return Option.empty(); } private static Path getNthParent(Path path, int n) { 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 index c0ad8b14794bd..f7d45db8b9a3f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -108,14 +108,15 @@ public class HoodieMetadataPayload implements HoodieRecordPayload recordOpt) { } else { columnStatMetadata = HoodieMetadataColumnStats.newBuilder() .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME)) + .setColumnName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME)) .setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)) .setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)) .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) @@ -508,6 +510,7 @@ public static Stream createColumnStatsRecords( HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), HoodieMetadataColumnStats.newBuilder() .setFileName(new Path(columnRangeMetadata.getFilePath()).getName()) + .setColumnName(columnRangeMetadata.getColumnName()) .setMinValue(columnRangeMetadata.getMinValue() == null ? null : columnRangeMetadata.getMinValue().toString()) .setMaxValue(columnRangeMetadata.getMaxValue() == null ? null : 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 index 841c5fc47e9ee..665eff3be407a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -56,14 +56,11 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable { String NON_PARTITIONED_NAME = "."; String EMPTY_PARTITION_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 for the given Dataset identified by base-path */ static String getMetadataTableBasePath(String dataTableBasePath) { - return dataTableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH; + return dataTableBasePath + Path.SEPARATOR + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH; } /** @@ -72,7 +69,7 @@ static String getMetadataTableBasePath(String dataTableBasePath) { */ static String getDataTableBasePathFromMetadataTable(String metadataTableBasePath) { checkArgument(isMetadataTable(metadataTableBasePath)); - return metadataTableBasePath.substring(0, metadataTableBasePath.lastIndexOf(METADATA_TABLE_REL_PATH) - 1); + return metadataTableBasePath.substring(0, metadataTableBasePath.lastIndexOf(HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH) - 1); } /** @@ -84,7 +81,7 @@ static boolean isMetadataTable(String basePath) { if (basePath.endsWith(Path.SEPARATOR)) { basePath = basePath.substring(0, basePath.length() - 1); } - return basePath.endsWith(METADATA_TABLE_REL_PATH); + return basePath.endsWith(HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); } static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, 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 index 1a3739df2f8ec..f0388cca14130 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -927,16 +927,13 @@ private static Stream getColumnStats(final String partitionPath, f final String fileName = filePathWithPartition.substring(offset); if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { - List> columnRangeMetadataList = new ArrayList<>(); final Path fullFilePath = new Path(datasetMetaClient.getBasePath(), filePathWithPartition); + List> columnRangeMetadataList; if (!isDeleted) { - try { - columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata( - datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex); - } catch (Exception e) { - LOG.error("Failed to read column stats for " + fullFilePath, e); - } + columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata( + datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex); } else { + // TODO we should delete records instead of stubbing them columnRangeMetadataList = columnsToIndex.stream().map(entry -> new HoodieColumnRangeMetadata(fileName, entry, null, null, 0, 0, 0, 0)) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java index ab573f0d89bbc..056f2121ce52b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java @@ -93,20 +93,28 @@ void getTablePathFromTablePath() throws IOException { @Test void getTablePathFromMetadataFolderPath() throws IOException { - Path metadataFolder = new Path(tablePath, HoodieTableMetaClient.METAFOLDER_NAME); - Option inferredTablePath = TablePathUtils.getTablePath(fs, metadataFolder); + Path metaFolder = new Path(tablePath, HoodieTableMetaClient.METAFOLDER_NAME); + Option inferredTablePath = TablePathUtils.getTablePath(fs, metaFolder); assertEquals(tablePath, inferredTablePath.get()); } @Test void getTablePathFromMetadataSubFolderPath() throws IOException { Path auxFolder = new Path(tablePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); - Option inferredTablePath = TablePathUtils.getTablePath(fs, auxFolder); - assertEquals(tablePath, inferredTablePath.get()); + assertEquals(tablePath, TablePathUtils.getTablePath(fs, auxFolder).get()); Path bootstrapIndexFolder = new Path(tablePath, HoodieTableMetaClient.BOOTSTRAP_INDEX_ROOT_FOLDER_PATH); - inferredTablePath = TablePathUtils.getTablePath(fs, bootstrapIndexFolder); - assertEquals(tablePath, inferredTablePath.get()); + assertEquals(tablePath, TablePathUtils.getTablePath(fs, bootstrapIndexFolder).get()); + + Path metadataTableFolder = new Path(tablePath, HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); + Path metadataTableMetaFolder = new Path(metadataTableFolder, HoodieTableMetaClient.METAFOLDER_NAME); + assertTrue(new File(metadataTableMetaFolder.toUri()).mkdirs()); + + assertEquals(metadataTableFolder, TablePathUtils.getTablePath(fs, metadataTableFolder).get()); + + Path metadataTablePartitionFolder = new Path(metadataTableFolder, "column_stats"); + assertTrue(new File(metadataTablePartitionFolder.toUri()).mkdir()); + assertEquals(metadataTableFolder, TablePathUtils.getTablePath(fs, metadataTablePartitionFolder).get()); } @Test diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 2b8dae255e3c4..5ae91dc46d96f 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop; +import org.apache.avro.Schema; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -47,7 +48,6 @@ import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.parquet.schema.MessageType; import javax.annotation.Nonnull; import java.io.IOException; @@ -278,10 +278,13 @@ protected static Option getHoodieVirtualKeyInfo(HoodieTabl TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); try { - MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema(); - return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), - tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()), - parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp()))); + Schema schema = tableSchemaResolver.getTableAvroSchema(); + return Option.of( + new HoodieVirtualKeyInfo( + tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp(), + schema.getField(tableConfig.getRecordKeyFieldProp()).pos(), + schema.getField(tableConfig.getPartitionFieldProp()).pos())); } catch (Exception exception) { throw new HoodieException("Fetching table schema failed with exception ", exception); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 1508babcbba97..7d558ec8cd10e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -18,21 +18,15 @@ package org.apache.hudi import org.apache.hadoop.fs.Path - import org.apache.hudi.DataSourceReadOptions._ -import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION} import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException -import org.apache.hudi.hadoop.HoodieROTablePathFilter - import org.apache.log4j.LogManager - -import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation} -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.hudi.streaming.HoodieStreamSource import org.apache.spark.sql.sources._ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetUtils.scala new file mode 100644 index 0000000000000..a6c689610b76d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDatasetUtils.scala @@ -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 + +import org.apache.spark.sql.DataFrame +import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK + +object HoodieDatasetUtils { + + /** + * Executes provided function while keeping provided [[DataFrame]] instance persisted for the + * duration of the execution + * + * @param df target [[DataFrame]] to be persisted + * @param level desired [[StorageLevel]] of the persistence + * @param f target function to be executed while [[DataFrame]] is kept persisted + * @tparam T return value of the target function + * @return execution outcome of the [[f]] function + */ + def withPersistence[T](df: DataFrame, level: StorageLevel = MEMORY_AND_DISK)(f: => T): T = { + df.persist(level) + try { + f + } finally { + df.unpersist() + } + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 9cdf5cc634ff9..6aa6fbb0eeaa9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -18,30 +18,31 @@ package org.apache.hudi import org.apache.hadoop.fs.{FileStatus, Path} - -import org.apache.hudi.HoodieFileIndex.getConfigProperties +import org.apache.hudi.HoodieDatasetUtils.withPersistence +import org.apache.hudi.HoodieFileIndex.{collectReferencedColumns, getConfigProperties} import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor} import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} - +import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata, MetadataPartitionType} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{And, Expression, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} -import org.apache.spark.sql.hudi.DataSkippingUtils.createColumnStatsIndexFilterExpr +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.hudi.DataSkippingUtils.translateIntoColumnStatsIndexFilterExpr import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.sql.{AnalysisException, Column, SparkSession} import org.apache.spark.unsafe.types.UTF8String +import java.text.SimpleDateFormat import scala.collection.JavaConverters._ -import scala.util.{Failure, Success, Try} import scala.util.control.NonFatal - -import java.text.SimpleDateFormat +import scala.util.{Failure, Success, Try} /** * A file index which support partition prune for hoodie snapshot and read-optimized query. @@ -84,7 +85,7 @@ case class HoodieFileIndex(spark: SparkSession, override def rootPaths: Seq[Path] = queryPaths.asScala - def enableDataSkipping(): Boolean = { + def isDataSkippingEnabled(): Boolean = { options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean } @@ -112,7 +113,6 @@ case class HoodieFileIndex(spark: SparkSession, * @return list of PartitionDirectory containing partition to base files mapping */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val convertedPartitionFilters = HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient, partitionFilters) @@ -121,18 +121,14 @@ case class HoodieFileIndex(spark: SparkSession, // - Col-Stats Index is present // - List of predicates (filters) is present val candidateFilesNamesOpt: Option[Set[String]] = - lookupCandidateFilesInColStatsIndex(dataFilters) match { + lookupCandidateFilesInMetadataTable(dataFilters) match { case Success(opt) => opt case Failure(e) => - if (e.isInstanceOf[AnalysisException]) { - logDebug("Failed to relay provided data filters to Z-index lookup", e) - } else { - logError("Failed to lookup candidate files in Z-index", e) - } + logError("Failed to lookup candidate files in Z-index", e) Option.empty } - logDebug(s"Overlapping candidate files (from Z-index): ${candidateFilesNamesOpt.getOrElse(Set.empty)}") + logDebug(s"Overlapping candidate files from Column Stats Index: ${candidateFilesNamesOpt.getOrElse(Set.empty)}") if (queryAsNonePartitionedTable) { // Read as Non-Partitioned table @@ -185,8 +181,8 @@ case class HoodieFileIndex(spark: SparkSession, /** * Computes pruned list of candidate base-files' names based on provided list of {@link dataFilters} - * conditions, by leveraging custom Column Statistics index (col-stats-index) bearing "min", "max", - * "num_nulls" statistics for all clustered columns. + * conditions, by leveraging Metadata Table's Column Statistics index (hereon referred as ColStats for brevity) + * bearing "min", "max", "num_nulls" statistics for all columns. * * NOTE: This method has to return complete set of candidate files, since only provided candidates will * ultimately be scanned as part of query execution. Hence, this method has to maintain the @@ -195,77 +191,102 @@ case class HoodieFileIndex(spark: SparkSession, * @param queryFilters list of original data filters passed down from querying engine * @return list of pruned (data-skipped) candidate base-files' names */ - private def lookupCandidateFilesInColStatsIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try { - val indexPath = metaClient.getColumnStatsIndexPath + private def lookupCandidateFilesInMetadataTable(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try { val fs = metaClient.getFs + val metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath) - if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || queryFilters.isEmpty) { - // scalastyle:off return - return Success(Option.empty) - // scalastyle:on return - } - - val completedCommits = getActiveTimeline.filterCompletedInstants().getInstants.iterator.asScala.toList.map(_.getTimestamp) - - // Collect all index tables present in `.zindex` folder - val candidateIndexTables = - fs.listStatus(new Path(indexPath)) - .filter(_.isDirectory) - .map(_.getPath.getName) - .filter(completedCommits.contains(_)) - .sortBy(x => x) - - if (candidateIndexTables.isEmpty) { - // scalastyle:off return - return Success(Option.empty) - // scalastyle:on return - } - - val dataFrameOpt = try { - Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString)) - } catch { - case t: Throwable => - logError("Failed to read col-stats index; skipping", t) - None + if (!isDataSkippingEnabled() || !fs.exists(new Path(metadataTablePath)) || queryFilters.isEmpty) { + Option.empty + } else { + val targetColStatsIndexColumns = Seq( + HoodieMetadataPayload.COLUMN_STATS_FIELD_FILE_NAME, + HoodieMetadataPayload.COLUMN_STATS_FIELD_MIN_VALUE, + HoodieMetadataPayload.COLUMN_STATS_FIELD_MAX_VALUE, + HoodieMetadataPayload.COLUMN_STATS_FIELD_NULL_COUNT) + + val requiredMetadataIndexColumns = + (targetColStatsIndexColumns :+ HoodieMetadataPayload.COLUMN_STATS_FIELD_COLUMN_NAME).map(colName => + s"${HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS}.${colName}") + + // Read Metadata Table's Column Stats Index into Spark's [[DataFrame]] + val metadataTableDF = spark.read.format("org.apache.hudi") + .load(s"$metadataTablePath/${MetadataPartitionType.COLUMN_STATS.getPartitionPath}") + + // TODO filter on (column, partition) prefix + val colStatsDF = metadataTableDF.where(col(HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS).isNotNull) + .select(requiredMetadataIndexColumns.map(col): _*) + + val queryReferencedColumns = collectReferencedColumns(spark, queryFilters, schema) + + // Persist DF to avoid re-computing column statistics unraveling + withPersistence(colStatsDF) { + // Metadata Table bears rows in the following format + // + // +---------------------------+------------+------------+------------+-------------+ + // | fileName | columnName | minValue | maxValue | num_nulls | + // +---------------------------+------------+------------+------------+-------------+ + // | one_base_file.parquet | A | 1 | 10 | 0 | + // | another_base_file.parquet | A | -10 | 0 | 5 | + // +---------------------------+------------+------------+------------+-------------+ + // + // While Data Skipping utils are expecting following (transposed) format, where per-column stats are + // essentially transposed (from rows to columns): + // + // +---------------------------+------------+------------+-------------+ + // | file | A_minValue | A_maxValue | A_num_nulls | + // +---------------------------+------------+------------+-------------+ + // | one_base_file.parquet | 1 | 10 | 0 | + // | another_base_file.parquet | -10 | 0 | 5 | + // +---------------------------+------------+------------+-------------+ + // + // NOTE: Column Stats Index might potentially contain statistics for many columns (if not all), while + // query at hand might only be referencing a handful of those. As such, we collect all the + // column references from the filtering expressions, and only transpose records corresponding to the + // columns referenced in those + val transposedColStatsDF = + queryReferencedColumns.map(colName => + colStatsDF.filter(col(HoodieMetadataPayload.COLUMN_STATS_FIELD_COLUMN_NAME).equalTo(colName)) + .select(targetColStatsIndexColumns.map(col): _*) + .withColumnRenamed(HoodieMetadataPayload.COLUMN_STATS_FIELD_NULL_COUNT, getNumNullsColumnNameFor(colName)) + .withColumnRenamed(HoodieMetadataPayload.COLUMN_STATS_FIELD_MIN_VALUE, getMinColumnNameFor(colName)) + .withColumnRenamed(HoodieMetadataPayload.COLUMN_STATS_FIELD_MAX_VALUE, getMaxColumnNameFor(colName)) + ) + .reduceLeft((left, right) => + left.join(right, usingColumn = HoodieMetadataPayload.COLUMN_STATS_FIELD_FILE_NAME)) + + // Persist DF to avoid re-computing column statistics unraveling + withPersistence(transposedColStatsDF) { + val indexSchema = transposedColStatsDF.schema + val indexFilter = + queryFilters.map(translateIntoColumnStatsIndexFilterExpr(_, indexSchema)) + .reduce(And) + + val allIndexedFileNames = + transposedColStatsDF.select(HoodieMetadataPayload.COLUMN_STATS_FIELD_FILE_NAME) + .collect() + .map(_.getString(0)) + .toSet + + val prunedCandidateFileNames = + transposedColStatsDF.where(new Column(indexFilter)) + .select(HoodieMetadataPayload.COLUMN_STATS_FIELD_FILE_NAME) + .collect() + .map(_.getString(0)) + .toSet + + // NOTE: Col-Stats Index isn't guaranteed to have complete set of statistics for every + // base-file: since it's bound to clustering, which could occur asynchronously + // at arbitrary point in time, and is not likely to be touching all of the base files. + // + // To close that gap, we manually compute the difference b/w all indexed (by col-stats-index) + // files and all outstanding base-files, and make sure that all base files not + // represented w/in the index are included in the output of this method + val notIndexedFileNames = lookupFileNamesMissingFromIndex(allIndexedFileNames) + + Some(prunedCandidateFileNames ++ notIndexedFileNames) + } + } } - - dataFrameOpt.map(df => { - val indexSchema = df.schema - val indexFilter = - queryFilters.map(createColumnStatsIndexFilterExpr(_, indexSchema)) - .reduce(And) - - logInfo(s"Index filter condition: $indexFilter") - - df.persist() - - val allIndexedFileNames = - df.select("file") - .collect() - .map(_.getString(0)) - .toSet - - val prunedCandidateFileNames = - df.where(new Column(indexFilter)) - .select("file") - .collect() - .map(_.getString(0)) - .toSet - - df.unpersist() - - // NOTE: Col-Stats Index isn't guaranteed to have complete set of statistics for every - // base-file: since it's bound to clustering, which could occur asynchronously - // at arbitrary point in time, and is not likely to be touching all of the base files. - // - // To close that gap, we manually compute the difference b/w all indexed (by col-stats-index) - // files and all outstanding base-files, and make sure that all base files not - // represented w/in the index are included in the output of this method - val notIndexedFileNames = - lookupFileNamesMissingFromIndex(allIndexedFileNames) - - prunedCandidateFileNames ++ notIndexedFileNames - }) } override def refresh(): Unit = super.refresh() @@ -282,6 +303,12 @@ case class HoodieFileIndex(spark: SparkSession, object HoodieFileIndex extends Logging { + private def collectReferencedColumns(spark: SparkSession, queryFilters: Seq[Expression], schema: StructType): Seq[String] = { + val resolver = spark.sessionState.analyzer.resolver + val refs = queryFilters.flatMap(_.references) + schema.fieldNames.filter { colName => refs.exists(r => resolver.apply(colName, r.name)) } + } + def getConfigProperties(spark: SparkSession, options: Map[String, String]) = { val sqlConf: SQLConf = spark.sessionState.conf val properties = new TypedProperties() @@ -331,6 +358,9 @@ object HoodieFileIndex extends Logging { } private def getQueryPath(options: Map[String, String]) = { - new Path(options.getOrElse("path", "'path' option required")) + new Path(options.get("path") match { + case Some(p) => p + case None => throw new IllegalArgumentException("'path' option required") + }) } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index d4c50b73fd402..74b8e10c4415b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -307,7 +307,7 @@ object SparkHoodieTableFileIndex { } private def deduceQueryType(configProperties: TypedProperties): HoodieTableQueryType = { - configProperties.asScala(QUERY_TYPE.key()) match { + configProperties.asScala(QUERY_TYPE.key) match { case QUERY_TYPE_SNAPSHOT_OPT_VAL => HoodieTableQueryType.SNAPSHOT case QUERY_TYPE_INCREMENTAL_OPT_VAL => HoodieTableQueryType.INCREMENTAL case QUERY_TYPE_READ_OPTIMIZED_OPT_VAL => HoodieTableQueryType.READ_OPTIMIZED diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala index e5d6f525bac4b..06b92e204fbed 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/DataSkippingUtils.scala @@ -17,39 +17,40 @@ package org.apache.spark.sql.hudi -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor} import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, StartsWith} -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.functions.col -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{StringType, StructType} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String -import scala.collection.JavaConverters._ - object DataSkippingUtils extends Logging { /** * Translates provided {@link filterExpr} into corresponding filter-expression for column-stats index index table * to filter out candidate files that would hold records matching the original filter * - * @param sourceFilterExpr source table's query's filter expression + * @param dataTableFilterExpr source table's query's filter expression * @param indexSchema index table schema * @return filter for column-stats index's table */ - def createColumnStatsIndexFilterExpr(sourceFilterExpr: Expression, indexSchema: StructType): Expression = { + def translateIntoColumnStatsIndexFilterExpr(dataTableFilterExpr: Expression, indexSchema: StructType): Expression = { + try { + createColumnStatsIndexFilterExprInternal(dataTableFilterExpr, indexSchema) + } catch { + case e: AnalysisException => + logDebug(s"Failed to translated provided data table filter expr into column stats one ($dataTableFilterExpr)", e) + throw e + } + } + + private def createColumnStatsIndexFilterExprInternal(dataTableFilterExpr: Expression, indexSchema: StructType): Expression = { // Try to transform original Source Table's filter expression into // Column-Stats Index filter expression - tryComposeIndexFilterExpr(sourceFilterExpr, indexSchema) match { + tryComposeIndexFilterExpr(dataTableFilterExpr, indexSchema) match { case Some(e) => e // NOTE: In case we can't transform source filter expression, we fallback // to {@code TrueLiteral}, to essentially avoid pruning any indexed files from scanning @@ -201,14 +202,14 @@ object DataSkippingUtils extends Logging { ) case or: Or => - val resLeft = createColumnStatsIndexFilterExpr(or.left, indexSchema) - val resRight = createColumnStatsIndexFilterExpr(or.right, indexSchema) + val resLeft = createColumnStatsIndexFilterExprInternal(or.left, indexSchema) + val resRight = createColumnStatsIndexFilterExprInternal(or.right, indexSchema) Option(Or(resLeft, resRight)) case and: And => - val resLeft = createColumnStatsIndexFilterExpr(and.left, indexSchema) - val resRight = createColumnStatsIndexFilterExpr(and.right, indexSchema) + val resLeft = createColumnStatsIndexFilterExprInternal(and.left, indexSchema) + val resRight = createColumnStatsIndexFilterExprInternal(and.right, indexSchema) Option(And(resLeft, resRight)) @@ -219,10 +220,10 @@ object DataSkippingUtils extends Logging { // case Not(And(left: Expression, right: Expression)) => - Option(createColumnStatsIndexFilterExpr(Or(Not(left), Not(right)), indexSchema)) + Option(createColumnStatsIndexFilterExprInternal(Or(Not(left), Not(right)), indexSchema)) case Not(Or(left: Expression, right: Expression)) => - Option(createColumnStatsIndexFilterExpr(And(Not(left), Not(right)), indexSchema)) + Option(createColumnStatsIndexFilterExprInternal(And(Not(left), Not(right)), indexSchema)) case _: Expression => None } @@ -259,34 +260,4 @@ object DataSkippingUtils extends Logging { throw new AnalysisException(s"convert reference to name failed, Found unsupported expression ${other}") } } - - def getIndexFiles(conf: Configuration, indexPath: String): Seq[FileStatus] = { - val basePath = new Path(indexPath) - basePath.getFileSystem(conf) - .listStatus(basePath).filter(f => f.getPath.getName.endsWith(".parquet")) - } - - /** - * read parquet files concurrently by local. - * this method is mush faster than spark - */ - def readParquetFile(spark: SparkSession, indexFiles: Seq[FileStatus], filters: Seq[Filter] = Nil, schemaOpts: Option[StructType] = None): Set[String] = { - val hadoopConf = spark.sparkContext.hadoopConfiguration - val partitionedFiles = indexFiles.map(f => PartitionedFile(InternalRow.empty, f.getPath.toString, 0, f.getLen)) - - val requiredSchema = new StructType().add("file", StringType, true) - val schema = schemaOpts.getOrElse(requiredSchema) - val parquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(spark - , schema , StructType(Nil), requiredSchema, filters, Map.empty, hadoopConf) - val results = new Array[Iterator[String]](partitionedFiles.size) - partitionedFiles.zipWithIndex.par.foreach { case (pf, index) => - val fileIterator = parquetReader(pf).asInstanceOf[Iterator[Any]] - val rows = fileIterator.flatMap(_ match { - case r: InternalRow => Seq(r) - case b: ColumnarBatch => b.rowIterator().asScala - }).map(r => r.getString(0)) - results(index) = rows - } - results.flatMap(f => f).toSet - } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala index ac866ba3e4d70..6b96472d4ce81 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala @@ -74,7 +74,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase { @MethodSource(Array("testBaseLookupFilterExpressionsSource", "testAdvancedLookupFilterExpressionsSource")) def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = { val resolvedExpr: Expression = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema) - val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) + val lookupFilter = DataSkippingUtils.translateIntoColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) val spark2 = spark import spark2.implicits._ @@ -94,7 +94,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase { @MethodSource(Array("testStringsLookupFilterExpressionsSource")) def testStringsLookupFilterExpressions(sourceExpr: Expression, input: Seq[IndexRow], output: Seq[String]): Unit = { val resolvedExpr = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema) - val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) + val lookupFilter = DataSkippingUtils.translateIntoColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) val spark2 = spark import spark2.implicits._ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index fa07c573f2725..899fc4cc2a6c1 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -18,43 +18,43 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration - +import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL} import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.client.HoodieJavaWriteClient import org.apache.hudi.client.common.HoodieJavaEngineContext import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.engine.EngineType -import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType} -import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.model.{HoodieRecord, HoodieTableQueryType, HoodieTableType} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.common.util.StringUtils.isNullOrEmpty -import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.TimestampType import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config -import org.apache.hudi.testutils.HoodieClientTestBase - +import org.apache.hudi.metadata.{HoodieTableMetadata, MetadataPartitionType} +import org.apache.hudi.testutils.{HoodieClientTestBase, SparkClientFunctionalTestHarness} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal} -import org.apache.spark.sql.execution.datasources.PartitionDirectory +import org.apache.spark.sql.execution.datasources.{NoopCache, PartitionDirectory} import org.apache.spark.sql.functions.{lit, struct} -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{IntegerType, StringType} import org.apache.spark.sql.{DataFrameWriter, Row, SaveMode, SparkSession} - import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{BeforeEach, Test} +import org.junit.jupiter.api.{BeforeEach, Tag, Test} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource, ValueSource} import java.util.Properties - - import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ +import scala.util.Random class TestHoodieFileIndex extends HoodieClientTestBase { @@ -333,6 +333,57 @@ class TestHoodieFileIndex extends HoodieClientTestBase { assert(fileIndex.getAllQueryPartitionPaths.get(0).path.equals("c")) } + @Test + def testDataSkippingWhileFileListing(): Unit = { + val r = new Random(0xDEED) + val tuples = for (i <- 1 to 1000) yield (i, 1000 - i, r.nextString(5), r.nextInt(4)) + + val _spark = spark + import _spark.implicits._ + val inputDF = tuples.toDF("id", "inv_id", "str", "rand") + + val opts = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + RECORDKEY_FIELD.key -> "id", + PRECOMBINE_FIELD.key -> "id", + HoodieMetadataConfig.ENABLE.key -> "true", + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true", + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS.key -> "true", + HoodieTableConfig.POPULATE_META_FIELDS.key -> "true" + ) + + inputDF.repartition(4) + .write + .format("hudi") + .options(opts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key, 100 * 1024) + .mode(SaveMode.Overwrite) + .save(basePath) + + metaClient = HoodieTableMetaClient.reload(metaClient) + + val props = Map[String, String]( + "path" -> basePath, + QUERY_TYPE.key -> QUERY_TYPE_SNAPSHOT_OPT_VAL, + DataSourceReadOptions.ENABLE_DATA_SKIPPING.key -> "true" + ) + + val fileIndex = HoodieFileIndex(spark, metaClient, Option.empty, props, NoopCache) + + val allFilesPartitions = fileIndex.listFiles(Seq(), Seq()) + assertEquals(10, allFilesPartitions.head.files.length) + + // We're selecting a single file that contains "id" == 1 row, which there should be + // strictly 1. Given that 1 is minimal possible value, Data Skipping should be able to + // truncate search space to just a single file + val dataFilter = EqualTo(AttributeReference("id", IntegerType, nullable = false)(), Literal(1)) + val filteredPartitions = fileIndex.listFiles(Seq(), Seq(dataFilter)) + assertEquals(1, filteredPartitions.head.files.length) + } + private def attribute(partition: String): AttributeReference = { AttributeReference(partition, StringType, true)() } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala index 818addaf87399..96728f620f370 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestLayoutOptimization.scala @@ -18,6 +18,7 @@ package org.apache.hudi.functional +import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings @@ -53,6 +54,8 @@ class TestLayoutOptimization extends HoodieClientTestBase { "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4", "hoodie.bulkinsert.shuffle.parallelism" -> "4", + HoodieMetadataConfig.ENABLE.key -> "true", + HoodieMetadataConfig.ENABLE_METADATA_INDEX_COLUMN_STATS.key -> "true", DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key", DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition", DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp", From d514570e909c7ca6eb1d41623029979ca3612806 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 16 Mar 2022 00:03:22 +0530 Subject: [PATCH 12/33] [HUDI-3633] Allow non-string values to be set in TypedProperties (#5045) * [HUDI-3633] Allow non-string values to be set in TypedProperties * Override getProperty to ignore instanceof string check --- .../hudi/common/config/TypedProperties.java | 16 ++++++++++++-- .../properties/TestTypedProperties.java | 22 +++++++++++++++++++ 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java index 2688e6454d814..09671ba2a3577 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Enumeration; import java.util.List; import java.util.Objects; import java.util.Properties; @@ -36,12 +37,23 @@ public TypedProperties() { public TypedProperties(Properties defaults) { if (Objects.nonNull(defaults)) { - for (String key : defaults.stringPropertyNames()) { - put(key, defaults.getProperty(key)); + for (Enumeration e = defaults.propertyNames(); e.hasMoreElements(); ) { + Object k = e.nextElement(); + Object v = defaults.get(k); + if (v != null) { + put(k, v); + } } } } + @Override + public String getProperty(String key) { + Object oval = super.get(key); + String sval = (oval != null) ? String.valueOf(oval) : null; + return ((sval == null) && (defaults != null)) ? defaults.getProperty(key) : sval; + } + private void checkKey(String property) { if (!containsKey(property)) { throw new IllegalArgumentException("Property " + property + " not found"); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java index f93627525d990..954b53651af5c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java @@ -83,5 +83,27 @@ public void testGetBoolean() { assertTrue(typedProperties.getBoolean("key1")); assertTrue(typedProperties.getBoolean("key1", false)); assertFalse(typedProperties.getBoolean("key2", false)); + // test getBoolean with non-string value for key2 + properties.put("key2", true); + typedProperties = new TypedProperties(properties); + assertTrue(typedProperties.getBoolean("key1", false)); + assertTrue(typedProperties.getBoolean("key2", false)); + // put non-string value in TypedProperties + typedProperties.put("key3", true); + assertTrue(typedProperties.getBoolean("key3", false)); + } + + @Test + public void testTypedPropertiesWithNonStringValue() { + Properties properties = new Properties(); + properties.put("key1", "1"); + properties.put("key2", 2); + + TypedProperties props = new TypedProperties(properties); + assertEquals(1, props.getInteger("key1")); + assertEquals(2, props.getInteger("key2")); + // put non-string value in TypedProperties + props.put("key2", 3); + assertEquals(3, props.getInteger("key2")); } } From 55dca969f9d25ea352dc0e343ef1f1e8539630fd Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 16 Mar 2022 03:56:37 +0800 Subject: [PATCH 13/33] [HUDI-3589] flink sync hive metadata supports table properties and serde properties (#4995) --- .../org/apache/hudi/configuration/FlinkOptions.java | 12 ++++++++++++ .../org/apache/hudi/sink/utils/HiveSyncContext.java | 2 ++ 2 files changed, 14 insertions(+) diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 34bab1285480c..90ed73a3deb83 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -675,6 +675,18 @@ private FlinkOptions() { .withDescription("INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n" + "Disabled by default for backward compatibility."); + public static final ConfigOption HIVE_SYNC_TABLE_PROPERTIES = ConfigOptions + .key("hive_sync.table_properties") + .stringType() + .noDefaultValue() + .withDescription("Additional properties to store with table, the data format is k1=v1\nk2=v2"); + + public static final ConfigOption HIVE_SYNC_TABLE_SERDE_PROPERTIES = ConfigOptions + .key("hive_sync.serde_properties") + .stringType() + .noDefaultValue() + .withDescription("Serde properties to hive table, the data format is k1=v1\nk2=v2"); + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index 1c051c8cd2300..768d36e0abe4b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -74,6 +74,8 @@ private static HiveSyncConfig buildSyncConfig(Configuration conf) { hiveSyncConfig.syncMode = conf.getString(FlinkOptions.HIVE_SYNC_MODE); hiveSyncConfig.hiveUser = conf.getString(FlinkOptions.HIVE_SYNC_USERNAME); hiveSyncConfig.hivePass = conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD); + hiveSyncConfig.tableProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES); + hiveSyncConfig.serdeProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES); hiveSyncConfig.jdbcUrl = conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL); hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractPartitionKeys(conf)); hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME); From 296a0e6bcf6f390288941e5ab8683390dcb6a788 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Tue, 15 Mar 2022 18:49:30 -0700 Subject: [PATCH 14/33] [HUDI-3588] Remove hudi-common and hudi-hadoop-mr jars in Presto Docker image (#4997) --- ...ker-compose_hadoop284_hive233_spark244.yml | 40 +++++++++---------- docker/hoodie/hadoop/pom.xml | 2 +- docker/hoodie/hadoop/prestobase/Dockerfile | 11 ++++- 3 files changed, 31 insertions(+), 22 deletions(-) diff --git a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml index 3c1acbdfe7714..086004f121e97 100644 --- a/docker/compose/docker-compose_hadoop284_hive233_spark244.yml +++ b/docker/compose/docker-compose_hadoop284_hive233_spark244.yml @@ -184,7 +184,7 @@ services: presto-coordinator-1: container_name: presto-coordinator-1 hostname: presto-coordinator-1 - image: apachehudi/hudi-hadoop_2.8.4-prestobase_0.268:latest + image: apachehudi/hudi-hadoop_2.8.4-prestobase_0.271:latest ports: - '8090:8090' environment: @@ -201,25 +201,25 @@ services: command: coordinator presto-worker-1: - container_name: presto-worker-1 - hostname: presto-worker-1 - image: apachehudi/hudi-hadoop_2.8.4-prestobase_0.268:latest - depends_on: ["presto-coordinator-1"] - environment: - - PRESTO_JVM_MAX_HEAP=512M - - PRESTO_QUERY_MAX_MEMORY=1GB - - PRESTO_QUERY_MAX_MEMORY_PER_NODE=256MB - - PRESTO_QUERY_MAX_TOTAL_MEMORY_PER_NODE=384MB - - PRESTO_MEMORY_HEAP_HEADROOM_PER_NODE=100MB - - TERM=xterm - links: - - "hivemetastore" - - "hiveserver" - - "hive-metastore-postgresql" - - "namenode" - volumes: - - ${HUDI_WS}:/var/hoodie/ws - command: worker + container_name: presto-worker-1 + hostname: presto-worker-1 + image: apachehudi/hudi-hadoop_2.8.4-prestobase_0.271:latest + depends_on: [ "presto-coordinator-1" ] + environment: + - PRESTO_JVM_MAX_HEAP=512M + - PRESTO_QUERY_MAX_MEMORY=1GB + - PRESTO_QUERY_MAX_MEMORY_PER_NODE=256MB + - PRESTO_QUERY_MAX_TOTAL_MEMORY_PER_NODE=384MB + - PRESTO_MEMORY_HEAP_HEADROOM_PER_NODE=100MB + - TERM=xterm + links: + - "hivemetastore" + - "hiveserver" + - "hive-metastore-postgresql" + - "namenode" + volumes: + - ${HUDI_WS}:/var/hoodie/ws + command: worker trino-coordinator-1: container_name: trino-coordinator-1 diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index de3bd3d57832d..b029abafa83e5 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -57,7 +57,7 @@ 2.4.4 2.3.3 2.8.4 - 0.268 + 0.271 368 1.4.13 true diff --git a/docker/hoodie/hadoop/prestobase/Dockerfile b/docker/hoodie/hadoop/prestobase/Dockerfile index 12b644aa06314..accedb94db3dc 100644 --- a/docker/hoodie/hadoop/prestobase/Dockerfile +++ b/docker/hoodie/hadoop/prestobase/Dockerfile @@ -22,7 +22,7 @@ ARG HADOOP_VERSION=2.8.4 ARG HIVE_VERSION=2.3.3 FROM apachehudi/hudi-hadoop_${HADOOP_VERSION}-base:latest as hadoop-base -ARG PRESTO_VERSION=0.268 +ARG PRESTO_VERSION=0.271 ENV PRESTO_VERSION ${PRESTO_VERSION} ENV PRESTO_HOME /opt/presto-server-${PRESTO_VERSION} @@ -79,6 +79,15 @@ RUN chmod +x /usr/local/bin/entrypoint.sh ADD target/ /var/hoodie/ws/docker/hoodie/hadoop/prestobase/target/ ENV HUDI_PRESTO_BUNDLE /var/hoodie/ws/docker/hoodie/hadoop/prestobase/target/hudi-presto-bundle.jar RUN cp ${HUDI_PRESTO_BUNDLE} ${PRESTO_HOME}/plugin/hive-hadoop2/ +# TODO: the latest master of Presto relies on hudi-presto-bundle, while current Presto releases +# rely on hudi-common and hudi-hadoop-mr 0.9.0, which are pulled in plugin/hive-hadoop2/ in the +# docker setup, making it hard to test the latest changes in Hudi due to class conflict. +# To get around the conflicts due to older Hudi jars below, they are removed for integration tests, +# so the hudi-presto-bundle build can be used solely for testing. This temporary logic must be +# removed once Presto has a new release depending on hudi-presto-bundle and we upgrade docker setup +# to that release version. +RUN rm ${PRESTO_HOME}/plugin/hive-hadoop2/hudi-common-* +RUN rm ${PRESTO_HOME}/plugin/hive-hadoop2/hudi-hadoop-mr-* VOLUME ["${PRESTO_LOG_DIR}"] From 91849c3d666a363c535610bc24be54c5d9aaa913 Mon Sep 17 00:00:00 2001 From: that's cool <1059023054@qq.com> Date: Wed, 16 Mar 2022 14:07:31 +0800 Subject: [PATCH 15/33] [HUDI-3607] Support backend switch in HoodieFlinkStreamer (#5032) * [HUDI-3607] Support backend switch in HoodieFlinkStreamer * [HUDI-3607] Support backend switch in HoodieFlinkStreamer 1. checkstyle fix * [HUDI-3607] Support backend switch in HoodieFlinkStreamer 1. change the msg --- hudi-flink/pom.xml | 6 +++ .../hudi/streamer/FlinkStreamerConfig.java | 7 ++++ .../hudi/streamer/HoodieFlinkStreamer.java | 4 +- .../hudi/util/FlinkStateBackendConverter.java | 41 +++++++++++++++++++ 4 files changed, 56 insertions(+), 2 deletions(-) create mode 100644 hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml index 27a4a0b453cb7..7338b23bf4b27 100644 --- a/hudi-flink/pom.xml +++ b/hudi-flink/pom.xml @@ -174,6 +174,12 @@ ${flink.version} provided + + org.apache.flink + flink-statebackend-rocksdb_${scala.binary.version} + ${flink.version} + provided + org.apache.parquet diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index 1d7111f495c58..c5d73036eda60 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -18,6 +18,8 @@ package org.apache.hudi.streamer; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; import org.apache.hudi.client.utils.OperationConverter; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -25,6 +27,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.keygen.constant.KeyGeneratorType; +import org.apache.hudi.util.FlinkStateBackendConverter; import org.apache.hudi.util.StreamerUtil; import com.beust.jcommander.Parameter; @@ -53,6 +56,10 @@ public class FlinkStreamerConfig extends Configuration { @Parameter(names = {"--flink-checkpoint-path"}, description = "Flink checkpoint path.") public String flinkCheckPointPath; + @Parameter(names = {"--flink-state-backend-type"}, description = "Flink state backend type, support only hashmap and rocksdb by now," + + " default hashmap.", converter = FlinkStateBackendConverter.class) + public StateBackend stateBackend = new HashMapStateBackend(); + @Parameter(names = {"--instant-retry-times"}, description = "Times to retry when latest instant has not completed.") public String instantRetryTimes = "10"; diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java index 851931f0d76ba..192de91d238a3 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java @@ -32,7 +32,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.formats.common.TimestampFormat; import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; -import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; @@ -61,8 +60,9 @@ public static void main(String[] args) throws Exception { // There can only be one checkpoint at one time. env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + env.setStateBackend(cfg.stateBackend); if (cfg.flinkCheckPointPath != null) { - env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath)); + env.getCheckpointConfig().setCheckpointStorage(cfg.flinkCheckPointPath); } TypedProperties kafkaProps = DFSPropertiesConfiguration.getGlobalProps(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java b/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java new file mode 100644 index 0000000000000..b46ab14e46384 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java @@ -0,0 +1,41 @@ +/* + * 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.util; + +import com.beust.jcommander.IStringConverter; +import com.beust.jcommander.ParameterException; +import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.hudi.exception.HoodieException; + +/** + * Converter that converts a string into Flink StateBackend. + */ +public class FlinkStateBackendConverter implements IStringConverter { + @Override + public StateBackend convert(String value) throws ParameterException { + switch (value) { + case "hashmap" : return new HashMapStateBackend(); + case "rocksdb" : return new EmbeddedRocksDBStateBackend(); + default: + throw new HoodieException(String.format("Unknown flink state backend %s.", value)); + } + } +} From 8ca9a54db016fc644070085fe2d596b0e5643e00 Mon Sep 17 00:00:00 2001 From: YueZhang <69956021+zhangyue19921010@users.noreply.github.com> Date: Thu, 17 Mar 2022 09:31:00 +0800 Subject: [PATCH 16/33] [Hudi-3376] Add an option to skip under deletion files for HoodieMetadataTableValidator (#4994) Co-authored-by: yuezhang --- .../HoodieMetadataTableValidator.java | 176 +++++++++++++++--- 1 file changed, 146 insertions(+), 30 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index be0044f275ca2..832d942c86afc 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.utilities; import org.apache.hudi.async.HoodieAsyncService; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.config.HoodieMetadataConfig; @@ -29,16 +30,21 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +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.util.CleanerUtils; 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.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; @@ -63,6 +69,7 @@ import java.util.Comparator; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -173,6 +180,9 @@ public static class Config implements Serializable { + "Can use --min-validate-interval-seconds to control validation frequency", required = false) public boolean continuous = false; + @Parameter(names = {"--skip-data-files-for-cleaning"}, description = "Skip to compare the data files which are under deletion by cleaner", required = false) + public boolean skipDataFilesForCleaning = false; + @Parameter(names = {"--validate-latest-file-slices"}, description = "Validate latest file slices for all partitions.", required = false) public boolean validateLatestFileSlices = false; @@ -230,6 +240,7 @@ public String toString() { + " --validate-all-column-stats " + validateAllColumnStats + ", \n" + " --validate-bloom-filters " + validateBloomFilters + ", \n" + " --continuous " + continuous + ", \n" + + " --skip-data-files-for-cleaning " + skipDataFilesForCleaning + ", \n" + " --ignore-failed " + ignoreFailed + ", \n" + " --min-validate-interval-seconds " + minValidateIntervalSeconds + ", \n" + " --parallelism " + parallelism + ", \n" @@ -252,6 +263,7 @@ public boolean equals(Object o) { Config config = (Config) o; return basePath.equals(config.basePath) && Objects.equals(continuous, config.continuous) + && Objects.equals(skipDataFilesForCleaning, config.skipDataFilesForCleaning) && Objects.equals(validateLatestFileSlices, config.validateLatestFileSlices) && Objects.equals(validateLatestBaseFiles, config.validateLatestBaseFiles) && Objects.equals(validateAllFileGroups, config.validateAllFileGroups) @@ -269,7 +281,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(basePath, continuous, validateLatestFileSlices, validateLatestBaseFiles, + return Objects.hash(basePath, continuous, skipDataFilesForCleaning, validateLatestFileSlices, validateLatestBaseFiles, validateAllFileGroups, validateAllColumnStats, validateBloomFilters, minValidateIntervalSeconds, parallelism, ignoreFailed, sparkMaster, sparkMemory, assumeDatePartitioning, propsFilePath, configs, help); } @@ -345,6 +357,33 @@ public void doMetadataTableValidation() { boolean finalResult = true; metaClient.reloadActiveTimeline(); String basePath = metaClient.getBasePath(); + Set baseFilesForCleaning = Collections.emptySet(); + + if (cfg.skipDataFilesForCleaning) { + HoodieTimeline inflightCleaningTimeline = metaClient.getActiveTimeline().getCleanerTimeline().filterInflights(); + + baseFilesForCleaning = inflightCleaningTimeline.getInstants().flatMap(instant -> { + try { + // convert inflight instant to requested and get clean plan + instant = new HoodieInstant(HoodieInstant.State.REQUESTED, instant.getAction(), instant.getTimestamp()); + HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(metaClient, instant); + + return cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().flatMap(cleanerFileInfoList -> { + return cleanerFileInfoList.stream().map(fileInfo -> { + return new Path(fileInfo.getFilePath()).getName(); + }); + }); + + } catch (IOException e) { + throw new HoodieIOException("Error reading cleaner metadata for " + instant); + } + // only take care of base files here. + }).filter(path -> { + String fileExtension = FSUtils.getFileExtension(path); + return HoodieFileFormat.BASE_FILE_EXTENSIONS.contains(fileExtension); + }).collect(Collectors.toSet()); + } + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); List allPartitions = validatePartitions(engineContext, basePath); HoodieMetadataValidationContext metadataTableBasedContext = @@ -352,9 +391,10 @@ public void doMetadataTableValidation() { HoodieMetadataValidationContext fsBasedContext = new HoodieMetadataValidationContext(engineContext, cfg, metaClient, false); + Set finalBaseFilesForCleaning = baseFilesForCleaning; List result = engineContext.parallelize(allPartitions, allPartitions.size()).map(partitionPath -> { try { - validateFilesInPartition(metadataTableBasedContext, fsBasedContext, partitionPath); + validateFilesInPartition(metadataTableBasedContext, fsBasedContext, partitionPath, finalBaseFilesForCleaning); LOG.info("Metadata table validation succeeded for " + partitionPath); return true; } catch (HoodieValidationException e) { @@ -410,42 +450,64 @@ private List validatePartitions(HoodieSparkEngineContext engineContext, * @param metadataTableBasedContext Validation context containing information based on metadata table * @param fsBasedContext Validation context containing information based on the file system * @param partitionPath Partition path String + * @param baseDataFilesForCleaning Base files for un-complete cleaner action */ private void validateFilesInPartition( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { + HoodieMetadataValidationContext fsBasedContext, String partitionPath, + Set baseDataFilesForCleaning) { if (cfg.validateLatestFileSlices) { - validateLatestFileSlices(metadataTableBasedContext, fsBasedContext, partitionPath); + validateLatestFileSlices(metadataTableBasedContext, fsBasedContext, partitionPath, baseDataFilesForCleaning); } if (cfg.validateLatestBaseFiles) { - validateLatestBaseFiles(metadataTableBasedContext, fsBasedContext, partitionPath); + validateLatestBaseFiles(metadataTableBasedContext, fsBasedContext, partitionPath, baseDataFilesForCleaning); } if (cfg.validateAllFileGroups) { - validateAllFileGroups(metadataTableBasedContext, fsBasedContext, partitionPath); + validateAllFileGroups(metadataTableBasedContext, fsBasedContext, partitionPath, baseDataFilesForCleaning); } if (cfg.validateAllColumnStats) { - validateAllColumnStats(metadataTableBasedContext, fsBasedContext, partitionPath); + validateAllColumnStats(metadataTableBasedContext, fsBasedContext, partitionPath, baseDataFilesForCleaning); } if (cfg.validateBloomFilters) { - validateBloomFilters(metadataTableBasedContext, fsBasedContext, partitionPath); + validateBloomFilters(metadataTableBasedContext, fsBasedContext, partitionPath, baseDataFilesForCleaning); } } private void validateAllFileGroups( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { - List allFileSlicesFromMeta = metadataTableBasedContext - .getSortedAllFileGroupList(partitionPath).stream() - .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) - .collect(Collectors.toList()); - List allFileSlicesFromFS = fsBasedContext - .getSortedAllFileGroupList(partitionPath).stream() - .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) - .collect(Collectors.toList()); + HoodieMetadataValidationContext fsBasedContext, + String partitionPath, + Set baseDataFilesForCleaning) { + + List allFileSlicesFromMeta; + List allFileSlicesFromFS; + + if (!baseDataFilesForCleaning.isEmpty()) { + List fileSlicesFromMeta = metadataTableBasedContext + .getSortedAllFileGroupList(partitionPath).stream() + .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) + .collect(Collectors.toList()); + List fileSlicesFromFS = fsBasedContext + .getSortedAllFileGroupList(partitionPath).stream() + .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) + .collect(Collectors.toList()); + + allFileSlicesFromMeta = filterFileSliceBasedOnInflightCleaning(fileSlicesFromMeta, baseDataFilesForCleaning); + allFileSlicesFromFS = filterFileSliceBasedOnInflightCleaning(fileSlicesFromFS, baseDataFilesForCleaning); + } else { + allFileSlicesFromMeta = metadataTableBasedContext + .getSortedAllFileGroupList(partitionPath).stream() + .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) + .collect(Collectors.toList()); + allFileSlicesFromFS = fsBasedContext + .getSortedAllFileGroupList(partitionPath).stream() + .flatMap(HoodieFileGroup::getAllFileSlices).sorted(new FileSliceComparator()) + .collect(Collectors.toList()); + } LOG.debug("All file slices from metadata: " + allFileSlicesFromMeta + ". For partitions " + partitionPath); LOG.debug("All file slices from direct listing: " + allFileSlicesFromFS + ". For partitions " + partitionPath); @@ -459,10 +521,20 @@ private void validateAllFileGroups( */ private void validateLatestBaseFiles( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { + HoodieMetadataValidationContext fsBasedContext, + String partitionPath, + Set baseDataFilesForCleaning) { - List latestFilesFromMetadata = metadataTableBasedContext.getSortedLatestBaseFileList(partitionPath); - List latestFilesFromFS = fsBasedContext.getSortedLatestBaseFileList(partitionPath); + List latestFilesFromMetadata; + List latestFilesFromFS; + + if (!baseDataFilesForCleaning.isEmpty()) { + latestFilesFromMetadata = filterBaseFileBasedOnInflightCleaning(metadataTableBasedContext.getSortedLatestBaseFileList(partitionPath), baseDataFilesForCleaning); + latestFilesFromFS = filterBaseFileBasedOnInflightCleaning(fsBasedContext.getSortedLatestBaseFileList(partitionPath), baseDataFilesForCleaning); + } else { + latestFilesFromMetadata = metadataTableBasedContext.getSortedLatestBaseFileList(partitionPath); + latestFilesFromFS = fsBasedContext.getSortedLatestBaseFileList(partitionPath); + } LOG.debug("Latest base file from metadata: " + latestFilesFromMetadata + ". For partitions " + partitionPath); LOG.debug("Latest base file from direct listing: " + latestFilesFromFS + ". For partitions " + partitionPath); @@ -483,10 +555,19 @@ private void validateLatestBaseFiles( */ private void validateLatestFileSlices( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { - - List latestFileSlicesFromMetadataTable = metadataTableBasedContext.getSortedLatestFileSliceList(partitionPath); - List latestFileSlicesFromFS = fsBasedContext.getSortedLatestFileSliceList(partitionPath); + HoodieMetadataValidationContext fsBasedContext, + String partitionPath, + Set baseDataFilesForCleaning) { + List latestFileSlicesFromMetadataTable; + List latestFileSlicesFromFS; + + if (!baseDataFilesForCleaning.isEmpty()) { + latestFileSlicesFromMetadataTable = filterFileSliceBasedOnInflightCleaning(metadataTableBasedContext.getSortedLatestFileSliceList(partitionPath), baseDataFilesForCleaning); + latestFileSlicesFromFS = filterFileSliceBasedOnInflightCleaning(fsBasedContext.getSortedLatestFileSliceList(partitionPath), baseDataFilesForCleaning); + } else { + latestFileSlicesFromMetadataTable = metadataTableBasedContext.getSortedLatestFileSliceList(partitionPath); + latestFileSlicesFromFS = fsBasedContext.getSortedLatestFileSliceList(partitionPath); + } LOG.debug("Latest file list from metadata: " + latestFileSlicesFromMetadataTable + ". For partition " + partitionPath); LOG.debug("Latest file list from direct listing: " + latestFileSlicesFromFS + ". For partition " + partitionPath); @@ -495,11 +576,31 @@ private void validateLatestFileSlices( LOG.info("Validation of getLatestFileSlices succeeded for partition " + partitionPath); } + private List filterFileSliceBasedOnInflightCleaning(List sortedLatestFileSliceList, Set baseDataFilesForCleaning) { + return sortedLatestFileSliceList.stream() + .filter(fileSlice -> { + if (!fileSlice.getBaseFile().isPresent()) { + return true; + } else { + return !baseDataFilesForCleaning.contains(fileSlice.getBaseFile().get().getFileName()); + } + }).collect(Collectors.toList()); + } + + private List filterBaseFileBasedOnInflightCleaning(List sortedBaseFileList, Set baseDataFilesForCleaning) { + return sortedBaseFileList.stream() + .filter(baseFile -> { + return !baseDataFilesForCleaning.contains(baseFile.getFileName()); + }).collect(Collectors.toList()); + } + private void validateAllColumnStats( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { - List latestBaseFilenameList = fsBasedContext.getSortedLatestBaseFileList(partitionPath) - .stream().map(BaseFile::getFileName).collect(Collectors.toList()); + HoodieMetadataValidationContext fsBasedContext, + String partitionPath, + Set baseDataFilesForCleaning) { + + List latestBaseFilenameList = getLatestBaseFileNames(fsBasedContext, partitionPath, baseDataFilesForCleaning); List> metadataBasedColStats = metadataTableBasedContext .getSortedColumnStatsList(partitionPath, latestBaseFilenameList); List> fsBasedColStats = fsBasedContext @@ -512,9 +613,11 @@ private void validateAllColumnStats( private void validateBloomFilters( HoodieMetadataValidationContext metadataTableBasedContext, - HoodieMetadataValidationContext fsBasedContext, String partitionPath) { - List latestBaseFilenameList = fsBasedContext.getSortedLatestBaseFileList(partitionPath) - .stream().map(BaseFile::getFileName).collect(Collectors.toList()); + HoodieMetadataValidationContext fsBasedContext, + String partitionPath, + Set baseDataFilesForCleaning) { + + List latestBaseFilenameList = getLatestBaseFileNames(fsBasedContext, partitionPath, baseDataFilesForCleaning); List metadataBasedBloomFilters = metadataTableBasedContext .getSortedBloomFilterList(partitionPath, latestBaseFilenameList); List fsBasedBloomFilters = fsBasedContext @@ -525,6 +628,19 @@ private void validateBloomFilters( LOG.info("Validation of bloom filters succeeded for partition " + partitionPath); } + private List getLatestBaseFileNames(HoodieMetadataValidationContext fsBasedContext, String partitionPath, Set baseDataFilesForCleaning) { + List latestBaseFilenameList; + if (!baseDataFilesForCleaning.isEmpty()) { + List sortedLatestBaseFileList = fsBasedContext.getSortedLatestBaseFileList(partitionPath); + latestBaseFilenameList = filterBaseFileBasedOnInflightCleaning(sortedLatestBaseFileList, baseDataFilesForCleaning) + .stream().map(BaseFile::getFileName).collect(Collectors.toList()); + } else { + latestBaseFilenameList = fsBasedContext.getSortedLatestBaseFileList(partitionPath) + .stream().map(BaseFile::getFileName).collect(Collectors.toList()); + } + return latestBaseFilenameList; + } + private void validate( List infoListFromMetadataTable, List infoListFromFS, String partitionPath, String label) { if (infoListFromMetadataTable.size() != infoListFromFS.size() From 95e6e538109af9fe60aa99219e4aa1d7ce9511e2 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 16 Mar 2022 22:25:04 -0700 Subject: [PATCH 17/33] [HUDI-3404] Automatically adjust write configs based on metadata table and write concurrency mode (#4975) --- .../apache/hudi/config/HoodieWriteConfig.java | 68 ++++- .../hudi/config/TestHoodieWriteConfig.java | 280 +++++++++++++----- 2 files changed, 272 insertions(+), 76 deletions(-) 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 4202cbda72765..b2e8b999be513 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 @@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; @@ -66,6 +67,8 @@ import org.apache.hudi.table.storage.HoodieStorageLayout; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; import org.apache.orc.CompressionKind; import org.apache.parquet.hadoop.metadata.CompressionCodecName; @@ -93,6 +96,7 @@ + "higher level frameworks (e.g Spark datasources, Flink sink) and utilities (e.g DeltaStreamer).") public class HoodieWriteConfig extends HoodieConfig { + private static final Logger LOG = LogManager.getLogger(HoodieWriteConfig.class); private static final long serialVersionUID = 0L; // This is a constant as is should never be changed via config (will invalidate previous commits) @@ -903,6 +907,11 @@ public String getTableName() { return getString(TBL_NAME); } + public HoodieTableType getTableType() { + return HoodieTableType.valueOf(getStringOrDefault( + HoodieTableConfig.TYPE, HoodieTableConfig.TYPE.defaultValue().name()).toUpperCase()); + } + public String getPreCombineField() { return getString(PRECOMBINE_FIELD_NAME); } @@ -1930,7 +1939,9 @@ public Boolean areAnyTableServicesExecutedInline() { * @return True if any table services are configured to run async, false otherwise. */ public Boolean areAnyTableServicesAsync() { - return isAsyncClusteringEnabled() || !inlineCompactionEnabled() || isAsyncClean() || isAsyncArchive(); + return isAsyncClusteringEnabled() + || (getTableType() == HoodieTableType.MERGE_ON_READ && !inlineCompactionEnabled()) + || isAsyncClean() || isAsyncArchive(); } public Boolean areAnyTableServicesScheduledInline() { @@ -2390,19 +2401,56 @@ protected void setDefaults() { HoodieLayoutConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); - // Async table services can update the metadata table and a lock provider is - // needed to guard against any concurrent table write operations. If user has - // not configured any lock provider, let's use the InProcess lock provider. + autoAdjustConfigsForConcurrencyMode(); + } + + private void autoAdjustConfigsForConcurrencyMode() { + boolean isMetadataTableEnabled = writeConfig.getBoolean(HoodieMetadataConfig.ENABLE); final TypedProperties writeConfigProperties = writeConfig.getProps(); final boolean isLockProviderPropertySet = writeConfigProperties.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME) || writeConfigProperties.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP); + if (!isLockConfigSet) { HoodieLockConfig.Builder lockConfigBuilder = HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()); - if (!isLockProviderPropertySet && writeConfig.areAnyTableServicesAsync()) { - lockConfigBuilder.withLockProvider(InProcessLockProvider.class); - } writeConfig.setDefault(lockConfigBuilder.build()); } + + if (isMetadataTableEnabled) { + // When metadata table is enabled, optimistic concurrency control must be used for + // single writer with async table services. + // Async table services can update the metadata table and a lock provider is + // needed to guard against any concurrent table write operations. If user has + // not configured any lock provider, let's use the InProcess lock provider. + boolean areTableServicesEnabled = writeConfig.areTableServicesEnabled(); + boolean areAsyncTableServicesEnabled = writeConfig.areAnyTableServicesAsync(); + + if (!isLockProviderPropertySet && areTableServicesEnabled && areAsyncTableServicesEnabled) { + // This is targeted at Single writer with async table services + // If user does not set the lock provider, likely that the concurrency mode is not set either + // Override the configs for metadata table + writeConfig.setValue(WRITE_CONCURRENCY_MODE.key(), + WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value()); + writeConfig.setValue(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), + InProcessLockProvider.class.getName()); + LOG.info(String.format("Automatically set %s=%s and %s=%s since user has not set the " + + "lock provider for single writer with async table services", + WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value(), + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getName())); + } + } + + // We check if "hoodie.cleaner.policy.failed.writes" + // is properly set to LAZY for optimistic concurrency control + String writeConcurrencyMode = writeConfig.getString(WRITE_CONCURRENCY_MODE); + if (WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value() + .equalsIgnoreCase(writeConcurrencyMode)) { + // In this case, we assume that the user takes care of setting the lock provider used + writeConfig.setValue(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + HoodieFailedWritesCleaningPolicy.LAZY.name()); + LOG.info(String.format("Automatically set %s=%s since optimistic concurrency control is used", + HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY.key(), + HoodieFailedWritesCleaningPolicy.LAZY.name())); + } } private void validate() { @@ -2411,9 +2459,9 @@ private void validate() { new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); Objects.requireNonNull(writeConfig.getString(BASE_PATH)); if (writeConfig.getString(WRITE_CONCURRENCY_MODE) - .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())) { - ValidationUtils.checkArgument(writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY) - != HoodieFailedWritesCleaningPolicy.EAGER.name(), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY"); + .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value())) { + ValidationUtils.checkArgument(!writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY) + .equals(HoodieFailedWritesCleaningPolicy.EAGER.name()), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY"); } } 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 2c3ae98c6e6be..778bef7324bde 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 @@ -21,20 +21,26 @@ import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.marker.MarkerType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig.Builder; import org.apache.hudi.index.HoodieIndex; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.Map; @@ -44,9 +50,11 @@ import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; import static org.apache.hudi.config.HoodieCompactionConfig.ASYNC_CLEAN; import static org.apache.hudi.config.HoodieCompactionConfig.AUTO_CLEAN; +import static org.apache.hudi.config.HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT; +import static org.apache.hudi.config.HoodieWriteConfig.TABLE_SERVICES_ENABLED; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieWriteConfig { @@ -114,89 +122,195 @@ public void testDefaultMarkersTypeAccordingToEngineType() { EngineType.JAVA, MarkerType.DIRECT)); } - @Test - public void testDefaultLockProviderWhenAsyncServicesEnabled() { + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testAutoConcurrencyConfigAdjustmentWithTableServices(HoodieTableType tableType) { final String inProcessLockProviderClassName = InProcessLockProvider.class.getCanonicalName(); - - // Any async clustering enabled should use InProcess lock provider - // as default when no other lock provider is set. - + // With metadata table enabled by default, any async table service enabled should + // use InProcess lock provider as default when no other lock provider is set. // 1. Async clustering - HoodieWriteConfig writeConfig = createWriteConfig(new HashMap() { - { - put(ASYNC_CLUSTERING_ENABLE.key(), "true"); - put(INLINE_COMPACT.key(), "true"); - put(AUTO_CLEAN.key(), "true"); - put(ASYNC_CLEAN.key(), "false"); - } - }); - assertTrue(writeConfig.areAnyTableServicesAsync()); - assertEquals(inProcessLockProviderClassName, writeConfig.getLockProviderClass()); + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(ASYNC_CLUSTERING_ENABLE.key(), "true"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "false"); + } + }), true, true, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, + HoodieFailedWritesCleaningPolicy.LAZY, inProcessLockProviderClassName); // 2. Async clean - writeConfig = createWriteConfig(new HashMap() { - { - put(ASYNC_CLUSTERING_ENABLE.key(), "false"); - put(INLINE_COMPACT.key(), "true"); - put(AUTO_CLEAN.key(), "true"); - put(ASYNC_CLEAN.key(), "true"); - } - }); - assertTrue(writeConfig.areAnyTableServicesAsync()); - assertEquals(inProcessLockProviderClassName, writeConfig.getLockProviderClass()); + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(ASYNC_CLUSTERING_ENABLE.key(), "false"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "true"); + } + }), true, true, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, + HoodieFailedWritesCleaningPolicy.LAZY, inProcessLockProviderClassName); - // 3. Async compaction - writeConfig = createWriteConfig(new HashMap() { - { - put(ASYNC_CLUSTERING_ENABLE.key(), "false"); - put(INLINE_COMPACT.key(), "false"); - put(AUTO_CLEAN.key(), "true"); - put(ASYNC_CLEAN.key(), "false"); - } - }); - assertTrue(writeConfig.areAnyTableServicesAsync()); - assertEquals(inProcessLockProviderClassName, writeConfig.getLockProviderClass()); + // 3. Async compaction configured + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(ASYNC_CLUSTERING_ENABLE.key(), "false"); + put(INLINE_COMPACT.key(), "false"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "false"); + } + }), true, + tableType == HoodieTableType.MERGE_ON_READ, + tableType == HoodieTableType.MERGE_ON_READ + ? WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL + : WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + tableType == HoodieTableType.MERGE_ON_READ + ? HoodieFailedWritesCleaningPolicy.LAZY + : HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + tableType == HoodieTableType.MERGE_ON_READ + ? inProcessLockProviderClassName + : HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); // 4. All inline services - writeConfig = createWriteConfig(new HashMap() { - { - put(ASYNC_CLUSTERING_ENABLE.key(), "false"); - put(INLINE_COMPACT.key(), "true"); - put(AUTO_CLEAN.key(), "true"); - put(ASYNC_CLEAN.key(), "false"); - } - }); - assertFalse(writeConfig.areAnyTableServicesAsync()); - assertTrue(writeConfig.areAnyTableServicesExecutedInline()); - assertEquals(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue(), writeConfig.getLockProviderClass()); + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(ASYNC_CLUSTERING_ENABLE.key(), "false"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "false"); + } + }), Option.of(true), Option.of(false), Option.of(true), + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); + } - // 5. User override for the lock provider should always take the precedence - writeConfig = HoodieWriteConfig.newBuilder() + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testAutoConcurrencyConfigAdjustmentWithUserConfigs(HoodieTableType tableType) { + // 1. User override for the lock provider should always take the precedence + TypedProperties properties = new TypedProperties(); + properties.setProperty(HoodieTableConfig.TYPE.key(), tableType.name()); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() .withPath("/tmp") .withLockConfig(HoodieLockConfig.newBuilder() .withLockProvider(FileSystemBasedLockProviderTestClass.class) .build()) - .build(); - assertEquals(FileSystemBasedLockProviderTestClass.class.getName(), writeConfig.getLockProviderClass()); - - // 6. User can set the lock provider via properties - TypedProperties properties = new TypedProperties(); - properties.setProperty(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), ZookeeperBasedLockProvider.class.getName()); - writeConfig = HoodieWriteConfig.newBuilder() - .withPath("/tmp") .withProperties(properties) .build(); - assertEquals(ZookeeperBasedLockProvider.class.getName(), writeConfig.getLockProviderClass()); + verifyConcurrencyControlRelatedConfigs(writeConfig, + true, tableType == HoodieTableType.MERGE_ON_READ, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + FileSystemBasedLockProviderTestClass.class.getName()); + + // 2. User can set the lock provider via properties + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(ASYNC_CLUSTERING_ENABLE.key(), "false"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "true"); + put(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), + ZookeeperBasedLockProvider.class.getName()); + } + }), true, true, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + ZookeeperBasedLockProvider.class.getName()); - // Default config should have default lock provider - writeConfig = createWriteConfig(Collections.emptyMap()); - if (!writeConfig.areAnyTableServicesAsync()) { - assertEquals(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue(), writeConfig.getLockProviderClass()); + // 3. Default config should have default lock provider + writeConfig = createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + } + }); + if (writeConfig.areAnyTableServicesAsync()) { + verifyConcurrencyControlRelatedConfigs(writeConfig, + true, true, + WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, + HoodieFailedWritesCleaningPolicy.LAZY, + InProcessLockProvider.class.getName()); } else { - assertEquals(inProcessLockProviderClassName, writeConfig.getLockProviderClass()); + verifyConcurrencyControlRelatedConfigs(writeConfig, + true, false, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); } } + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testAutoConcurrencyConfigAdjustmentWithNoTableService(HoodieTableType tableType) { + // 1. No table service, concurrency control configs should not be overwritten + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(TABLE_SERVICES_ENABLED.key(), "false"); + } + }), false, tableType == HoodieTableType.MERGE_ON_READ, + WriteConcurrencyMode.fromValue(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); + + // 2. No table service, with optimistic concurrency control, + // failed write clean policy should be updated accordingly + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(TABLE_SERVICES_ENABLED.key(), "false"); + put(WRITE_CONCURRENCY_MODE.key(), + WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value()); + put(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), + FileSystemBasedLockProviderTestClass.class.getName()); + } + }), false, tableType == HoodieTableType.MERGE_ON_READ, + WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, + HoodieFailedWritesCleaningPolicy.LAZY, + FileSystemBasedLockProviderTestClass.class.getName()); + } + + @ParameterizedTest + @EnumSource(HoodieTableType.class) + public void testAutoConcurrencyConfigAdjustmentWithMetadataTableDisabled(HoodieTableType tableType) { + // 1. Metadata table disabled, with async table services, concurrency control configs + // should not be changed + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(HoodieTableConfig.TYPE.key(), tableType.name()); + put(HoodieMetadataConfig.ENABLE.key(), "false"); + put(ASYNC_CLUSTERING_ENABLE.key(), "true"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "false"); + } + }), true, true, + WriteConcurrencyMode.fromValue(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); + + // 2. Metadata table disabled, with optimistic concurrency control, + // failed write clean policy should be updated accordingly + verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { + { + put(ASYNC_CLUSTERING_ENABLE.key(), "true"); + put(INLINE_COMPACT.key(), "true"); + put(AUTO_CLEAN.key(), "true"); + put(ASYNC_CLEAN.key(), "false"); + put(WRITE_CONCURRENCY_MODE.key(), + WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value()); + put(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), + FileSystemBasedLockProviderTestClass.class.getName()); + } + }), true, true, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, + HoodieFailedWritesCleaningPolicy.LAZY, FileSystemBasedLockProviderTestClass.class.getName()); + } + private HoodieWriteConfig createWriteConfig(Map configs) { final Properties properties = new Properties(); configs.forEach(properties::setProperty); @@ -253,4 +367,38 @@ private Map constructConfigMap( mapping.put(k3, v3); return mapping; } + + private void verifyConcurrencyControlRelatedConfigs( + HoodieWriteConfig writeConfig, boolean expectedTableServicesEnabled, + boolean expectedAnyTableServicesAsync, + WriteConcurrencyMode expectedConcurrencyMode, + HoodieFailedWritesCleaningPolicy expectedCleanPolicy, + String expectedLockProviderName) { + verifyConcurrencyControlRelatedConfigs(writeConfig, Option.of(expectedTableServicesEnabled), + Option.of(expectedAnyTableServicesAsync), Option.empty(), expectedConcurrencyMode, + expectedCleanPolicy, expectedLockProviderName); + } + + private void verifyConcurrencyControlRelatedConfigs( + HoodieWriteConfig writeConfig, Option expectedTableServicesEnabled, + Option expectedAnyTableServicesAsync, + Option expectedAnyTableServicesExecutedInline, + WriteConcurrencyMode expectedConcurrencyMode, + HoodieFailedWritesCleaningPolicy expectedCleanPolicy, + String expectedLockProviderName) { + if (expectedTableServicesEnabled.isPresent()) { + assertEquals(expectedTableServicesEnabled.get(), writeConfig.areTableServicesEnabled()); + } + if (expectedAnyTableServicesAsync.isPresent()) { + assertEquals(expectedAnyTableServicesAsync.get(), writeConfig.areAnyTableServicesAsync()); + } + if (expectedAnyTableServicesExecutedInline.isPresent()) { + assertEquals(expectedAnyTableServicesExecutedInline.get(), + writeConfig.areAnyTableServicesExecutedInline()); + } + + assertEquals(expectedConcurrencyMode, writeConfig.getWriteConcurrencyMode()); + assertEquals(expectedCleanPolicy, writeConfig.getFailedWritesCleanPolicy()); + assertEquals(expectedLockProviderName, writeConfig.getLockProviderClass()); + } } From 5ba2d9ab2f9f22cf81968a68e0303d750a2bd9e9 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Wed, 16 Mar 2022 22:28:11 -0700 Subject: [PATCH 18/33] [HUDI-3494] Consider triggering condition of MOR compaction during archival (#4974) --- .../hudi/client/HoodieTimelineArchiver.java | 39 +++- .../ScheduleCompactionActionExecutor.java | 29 ++- .../hudi/io/TestHoodieTimelineArchiver.java | 167 +++++++++++++++++- .../hudi/common/util/CompactionUtils.java | 83 ++++++++- .../hudi/common/util/TestCompactionUtils.java | 120 ++++++++++++- 5 files changed, 400 insertions(+), 38 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 15401c0292e14..66c89cfdc014a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; @@ -43,6 +44,7 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -52,6 +54,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -76,6 +79,7 @@ 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.LESSER_THAN; import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; /** @@ -395,6 +399,18 @@ private Stream getCommitInstantsToArchive() { // made after the first savepoint present. Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxInstantsToKeep) { + // For Merge-On-Read table, inline or async compaction is enabled + // We need to make sure that there are enough delta commits in the active timeline + // to trigger compaction scheduling, when the trigger strategy of compaction is + // NUM_COMMITS or NUM_AND_TIME. + Option oldestInstantToRetainForCompaction = + (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ + && (config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_COMMITS + || config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_AND_TIME)) + ? CompactionUtils.getOldestInstantToRetainForCompaction( + table.getActiveTimeline(), config.getInlineCompactDeltaCommitMax()) + : Option.empty(); + // Actually do the commits Stream instantToArchiveStream = commitTimeline.getInstants() .filter(s -> { @@ -405,14 +421,21 @@ private Stream getCommitInstantsToArchive() { return oldestPendingCompactionAndReplaceInstant .map(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) .orElse(true); - }); - // We need this to ensure that when multiple writers are performing conflict resolution, eligible instants don't - // get archived, i.e, instants after the oldestInflight are retained on the timeline - if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.LAZY) { - instantToArchiveStream = instantToArchiveStream.filter(s -> oldestInflightCommitInstant.map(instant -> - HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) - .orElse(true)); - } + }).filter(s -> { + // We need this to ensure that when multiple writers are performing conflict resolution, eligible instants don't + // get archived, i.e, instants after the oldestInflight are retained on the timeline + if (config.getFailedWritesCleanPolicy() == HoodieFailedWritesCleaningPolicy.LAZY) { + return oldestInflightCommitInstant.map(instant -> + HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) + .orElse(true); + } + return true; + }).filter(s -> + oldestInstantToRetainForCompaction.map(instantToRetain -> + HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN, instantToRetain.getTimestamp())) + .orElse(true) + ); + return instantToArchiveStream.limit(commitTimeline.countInstants() - minInstantsToKeep); } else { return Stream.empty(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java index 2627454fccb60..d3cc5660bc70a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; @@ -128,27 +129,25 @@ private HoodieCompactionPlan scheduleCompaction() { return new HoodieCompactionPlan(); } - private Pair getLatestDeltaCommitInfo() { - Option lastCompaction = table.getActiveTimeline().getCommitTimeline() - .filterCompletedInstants().lastInstant(); - HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline(); - - String latestInstantTs; - final int deltaCommitsSinceLastCompaction; - if (lastCompaction.isPresent()) { - latestInstantTs = lastCompaction.get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(latestInstantTs, Integer.MAX_VALUE).countInstants(); - } else { - latestInstantTs = deltaCommits.firstInstant().get().getTimestamp(); - deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(latestInstantTs, Integer.MAX_VALUE).countInstants(); + private Option> getLatestDeltaCommitInfo() { + Option> deltaCommitsInfo = + CompactionUtils.getDeltaCommitsSinceLatestCompaction(table.getActiveTimeline()); + if (deltaCommitsInfo.isPresent()) { + return Option.of(Pair.of( + deltaCommitsInfo.get().getLeft().countInstants(), + deltaCommitsInfo.get().getRight().getTimestamp())); } - return Pair.of(deltaCommitsSinceLastCompaction, latestInstantTs); + return Option.empty(); } private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) { boolean compactable; // get deltaCommitsSinceLastCompaction and lastCompactionTs - Pair latestDeltaCommitInfo = getLatestDeltaCommitInfo(); + Option> latestDeltaCommitInfoOption = getLatestDeltaCommitInfo(); + if (!latestDeltaCommitInfoOption.isPresent()) { + return false; + } + Pair latestDeltaCommitInfo = latestDeltaCommitInfoOption.get(); int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax(); int inlineCompactDeltaSecondsMax = config.getInlineCompactDeltaSecondsMax(); switch (compactionTriggerStrategy) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index 652dbcb155b0e..aafc538213738 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -44,6 +44,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.HoodieSparkTable; @@ -71,6 +72,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import static org.apache.hudi.common.testutils.HoodieTestUtils.createCompactionCommitInMetadataTable; @@ -125,9 +127,20 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, + int maxDeltaCommits, int maxDeltaCommitsMetadataTable, HoodieTableType tableType) throws Exception { - return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200); + return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, + maxDeltaCommits, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200); + } + + private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, + int minArchivalCommits, + int maxArchivalCommits, + int maxDeltaCommitsMetadataTable, + HoodieTableType tableType) throws Exception { + return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, + 5, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200); } private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, @@ -137,13 +150,14 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, boolean enableArchiveMerge, int archiveFilesBatch, long size) throws Exception { - return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, + return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, 5, maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE, enableArchiveMerge, archiveFilesBatch, size); } private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, + int maxDeltaCommits, int maxDeltaCommitsMetadataTable, HoodieTableType tableType, boolean enableArchiveMerge, @@ -153,6 +167,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits) + .withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits) .withArchiveMergeEnable(enableArchiveMerge) .withArchiveMergeFilesBatchSize(archiveFilesBatch) .withArchiveMergeSmallFileLimit(size) @@ -546,7 +561,7 @@ public void testArchiveRollbacksTestTable(boolean enableMetadata) throws Excepti @ParameterizedTest @ValueSource(booleans = {true, false}) public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) throws Exception { - HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2, + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2, 2, HoodieTableType.MERGE_ON_READ); // when max archival commits is set to 4, even after 7 commits, if there is an inflight compaction in the middle, archival should not kick in. @@ -946,6 +961,152 @@ public void testArchiveCommitsWithCompactionCommitInMetadataTableTimeline() thro } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testArchivalWithMaxDeltaCommitsGuaranteeForCompaction(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig( + enableMetadata, 2, 4, 8, 1, HoodieTableType.MERGE_ON_READ); + + // When max archival commits is set to 4, even after 8 delta commits, since the number of delta + // commits is still smaller than 8, the archival should not kick in. + // The archival should only kick in after the 9th delta commit + // instant "00000001" to "00000009" + for (int i = 1; i < 10; i++) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 + ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + + if (i <= 8) { + assertEquals(originalCommits, commitsAfterArchival); + } else { + assertEquals(1, originalCommits.size() - commitsAfterArchival.size()); + assertFalse(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "00000001"))); + IntStream.range(2, 10).forEach(j -> + assertTrue(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j)))); + } + } + + testTable.doCompaction("00000010", Arrays.asList("p1", "p2")); + + // instant "00000011" to "00000019" + for (int i = 1; i < 10; i++) { + testTable.doWriteOperation("0000001" + i, WriteOperationType.UPSERT, i == 1 + ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + + // first 9 delta commits before the completed compaction should be archived + IntStream.range(1, 10).forEach(j -> + assertFalse(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j)))); + + if (i == 1) { + assertEquals(8, originalCommits.size() - commitsAfterArchival.size()); + // instant from "00000011" should be in the active timeline + assertTrue(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000010"))); + assertTrue(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "00000011"))); + } else if (i < 8) { + assertEquals(originalCommits, commitsAfterArchival); + } else { + assertEquals(1, originalCommits.size() - commitsAfterArchival.size()); + assertFalse(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000010"))); + // i == 8 -> ["00000011", "00000018"] should be in the active timeline + // i == 9 -> ["00000012", "00000019"] should be in the active timeline + if (i == 9) { + assertFalse(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "00000011"))); + } + IntStream.range(i - 7, i + 1).forEach(j -> + assertTrue(commitsAfterArchival.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000001" + j)))); + } + } + } + + @Test + public void testArchivalAndCompactionInMetadataTable() throws Exception { + init(HoodieTableType.COPY_ON_WRITE); + // Test configs where metadata table has more aggressive archival configs than the compaction config + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .retainCommits(1).archiveCommitsWith(2, 4).build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(timelineServicePort).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true) + .withMaxNumDeltaCommitsBeforeCompaction(8) + .retainCommits(1).archiveCommitsWith(2, 4).build()) + .forTable("test-trip-table").build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + + HoodieTableMetaClient metadataTableMetaClient = HoodieTableMetaClient.builder() + .setConf(metaClient.getHadoopConf()) + .setBasePath(HoodieTableMetadata.getMetadataTableBasePath(basePath)) + .setLoadActiveTimelineOnLoad(true).build(); + + for (int i = 1; i <= 16; i++) { + testTable.doWriteOperation("000000" + String.format("%02d", i), WriteOperationType.UPSERT, + i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + // archival + archiveAndGetCommitsList(writeConfig); + + metadataTableMetaClient = HoodieTableMetaClient.reload(metadataTableMetaClient); + List metadataTableInstants = metadataTableMetaClient.getActiveTimeline() + .getCommitsTimeline().filterCompletedInstants().getInstants() + .collect(Collectors.toList()); + + if (i <= 7) { + // In the metadata table timeline, the first delta commit is "00000000000000" + // from metadata table init, delta commits "00000001" till "00000007" are added + // later on without archival or compaction + assertEquals(i + 1, metadataTableInstants.size()); + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "00000000000000"))); + IntStream.range(1, i + 1).forEach(j -> + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j)))); + } else if (i <= 14) { + // In the metadata table timeline, the first delta commit is "00000007001" + // from metadata table compaction, after archival, delta commits "00000008" + // till "00000014" are added later on without archival or compaction + assertEquals(i - 6, metadataTableInstants.size()); + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000007001"))); + IntStream.range(8, i + 1).forEach(j -> + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, + "000000" + String.format("%02d", j))))); + } else if (i == 15) { + // Only delta commits "00000008" till "00000015" are in the active timeline + assertEquals(8, metadataTableInstants.size()); + assertFalse(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000007001"))); + IntStream.range(8, 16).forEach(j -> + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, + "000000" + String.format("%02d", j))))); + } else { + // i == 16 + // Only commit "00000015001" and delta commit "00000016" are in the active timeline + assertEquals(2, metadataTableInstants.size()); + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000015001"))); + assertTrue(metadataTableInstants.contains( + new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "00000016"))); + } + } + } + private Pair, List> archiveAndGetCommitsList(HoodieWriteConfig writeConfig) throws IOException { metaClient.reloadActiveTimeline(); HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java index f3e4dc62837c1..14308d5df3b58 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroupId; 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; @@ -59,13 +60,13 @@ public class CompactionUtils { /** * Generate compaction operation from file-slice. * - * @param partitionPath Partition path - * @param fileSlice File Slice + * @param partitionPath Partition path + * @param fileSlice File Slice * @param metricsCaptureFunction Metrics Capture function * @return Compaction Operation */ public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice, - Option, Map>> metricsCaptureFunction) { + Option, Map>> metricsCaptureFunction) { HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder(); builder.setPartitionPath(partitionPath); builder.setFileId(fileSlice.getFileId()); @@ -87,12 +88,12 @@ public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, * Generate compaction plan from file-slices. * * @param partitionFileSlicePairs list of partition file-slice pairs - * @param extraMetadata Extra Metadata - * @param metricsCaptureFunction Metrics Capture function + * @param extraMetadata Extra Metadata + * @param metricsCaptureFunction Metrics Capture function */ public static HoodieCompactionPlan buildFromFileSlices(List> partitionFileSlicePairs, - Option> extraMetadata, - Option, Map>> metricsCaptureFunction) { + Option> extraMetadata, + Option, Map>> metricsCaptureFunction) { HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder(); extraMetadata.ifPresent(builder::setExtraMetadata); @@ -195,10 +196,76 @@ public static Stream getPendingCompactionInstantTimes(HoodieTableMetaClient metaClient) { return metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList()); } + + /** + * Returns a pair of (timeline containing the delta commits after the latest completed + * compaction commit, the completed compaction commit instant), if the latest completed + * compaction commit is present; a pair of (timeline containing all the delta commits, + * the first delta commit instant), if there is no completed compaction commit. + * + * @param activeTimeline Active timeline of a table. + * @return Pair of timeline containing delta commits and an instant. + */ + public static Option> getDeltaCommitsSinceLatestCompaction( + HoodieActiveTimeline activeTimeline) { + Option lastCompaction = activeTimeline.getCommitTimeline() + .filterCompletedInstants().lastInstant(); + HoodieTimeline deltaCommits = activeTimeline.getDeltaCommitTimeline(); + + HoodieInstant latestInstant; + if (lastCompaction.isPresent()) { + latestInstant = lastCompaction.get(); + // timeline containing the delta commits after the latest completed compaction commit, + // and the completed compaction commit instant + return Option.of(Pair.of(deltaCommits.findInstantsAfter( + latestInstant.getTimestamp(), Integer.MAX_VALUE), lastCompaction.get())); + } else { + if (deltaCommits.countInstants() > 0) { + latestInstant = deltaCommits.firstInstant().get(); + // timeline containing all the delta commits, and the first delta commit instant + return Option.of(Pair.of(deltaCommits.findInstantsAfterOrEquals( + latestInstant.getTimestamp(), Integer.MAX_VALUE), latestInstant)); + } else { + return Option.empty(); + } + } + } + + /** + * Gets the oldest instant to retain for MOR compaction. + * If there is no completed compaction, + * num delta commits >= "hoodie.compact.inline.max.delta.commits" + * If there is a completed compaction, + * num delta commits after latest completed compaction >= "hoodie.compact.inline.max.delta.commits" + * + * @param activeTimeline Active timeline of a table. + * @param maxDeltaCommits Maximum number of delta commits that trigger the compaction plan, + * i.e., "hoodie.compact.inline.max.delta.commits". + * @return the oldest instant to keep for MOR compaction. + */ + public static Option getOldestInstantToRetainForCompaction( + HoodieActiveTimeline activeTimeline, int maxDeltaCommits) { + Option> deltaCommitsInfoOption = + CompactionUtils.getDeltaCommitsSinceLatestCompaction(activeTimeline); + if (deltaCommitsInfoOption.isPresent()) { + Pair deltaCommitsInfo = deltaCommitsInfoOption.get(); + HoodieTimeline deltaCommitTimeline = deltaCommitsInfo.getLeft(); + int numDeltaCommits = deltaCommitTimeline.countInstants(); + if (numDeltaCommits < maxDeltaCommits) { + return Option.of(deltaCommitsInfo.getRight()); + } else { + // delta commits with the last one to keep + List instants = deltaCommitTimeline.getInstants() + .limit(numDeltaCommits - maxDeltaCommits + 1).collect(Collectors.toList()); + return Option.of(instants.get(instants.size() - 1)); + } + } + return Option.empty(); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java index e3c89a377384b..30abe48cb4e19 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java @@ -27,6 +27,9 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.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.versioning.compaction.CompactionPlanMigrator; import org.apache.hudi.common.testutils.CompactionTestUtils.DummyHoodieBaseFile; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; @@ -35,15 +38,20 @@ import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.hudi.common.testutils.CompactionTestUtils.createCompactionPlan; import static org.apache.hudi.common.testutils.CompactionTestUtils.scheduleCompaction; @@ -230,11 +238,95 @@ public void testGetAllPendingCompactionOperationsForEmptyCompactions() throws IO setupAndValidateCompactionOperations(metaClient, false, 0, 0, 0, 0); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGetDeltaCommitsSinceLatestCompaction(boolean hasCompletedCompaction) { + HoodieActiveTimeline timeline = prepareTimeline(hasCompletedCompaction); + Pair actual = + CompactionUtils.getDeltaCommitsSinceLatestCompaction(timeline).get(); + if (hasCompletedCompaction) { + Stream instants = actual.getLeft().getInstants(); + assertEquals( + Stream.of( + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) + .collect(Collectors.toList()), + actual.getLeft().getInstants().collect(Collectors.toList())); + assertEquals( + new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "06"), + actual.getRight()); + } else { + assertEquals( + Stream.of( + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "02"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "03"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "04"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "05"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) + .collect(Collectors.toList()), + actual.getLeft().getInstants().collect(Collectors.toList())); + assertEquals( + new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), + actual.getRight()); + } + } + + @Test + public void testGetDeltaCommitsSinceLatestCompactionWithEmptyDeltaCommits() { + HoodieActiveTimeline timeline = new MockHoodieActiveTimeline(); + assertEquals(Option.empty(), CompactionUtils.getDeltaCommitsSinceLatestCompaction(timeline)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGetOldestInstantToKeepForCompaction(boolean hasCompletedCompaction) { + HoodieActiveTimeline timeline = prepareTimeline(hasCompletedCompaction); + Option actual = CompactionUtils.getOldestInstantToRetainForCompaction(timeline, 20); + + if (hasCompletedCompaction) { + assertEquals(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "06"), actual.get()); + } else { + assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), actual.get()); + } + + actual = CompactionUtils.getOldestInstantToRetainForCompaction(timeline, 3); + assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), actual.get()); + + actual = CompactionUtils.getOldestInstantToRetainForCompaction(timeline, 2); + assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), actual.get()); + } + + @Test + public void testGetOldestInstantToKeepForCompactionWithEmptyDeltaCommits() { + HoodieActiveTimeline timeline = new MockHoodieActiveTimeline(); + assertEquals(Option.empty(), CompactionUtils.getOldestInstantToRetainForCompaction(timeline, 20)); + } + + private HoodieActiveTimeline prepareTimeline(boolean hasCompletedCompaction) { + if (hasCompletedCompaction) { + return new MockHoodieActiveTimeline( + Stream.of("01", "02", "03", "04", "05", "07", "08"), + Stream.of("06"), + Stream.of(Pair.of("09", HoodieTimeline.DELTA_COMMIT_ACTION))); + } else { + return new MockHoodieActiveTimeline( + Stream.of("01", "02", "03", "04", "05", "07", "08"), + Stream.empty(), + Stream.of( + Pair.of("06", HoodieTimeline.COMMIT_ACTION), + Pair.of("09", HoodieTimeline.DELTA_COMMIT_ACTION))); + } + } + /** * Validates if generated compaction plan matches with input file-slices. * * @param input File Slices with partition-path - * @param plan Compaction Plan + * @param plan Compaction Plan */ private void testFileSlicesCompactionPlanEquality(List> input, HoodieCompactionPlan plan) { assertEquals(input.size(), plan.getOperations().size(), "All file-slices present"); @@ -245,12 +337,12 @@ private void testFileSlicesCompactionPlanEquality(List> /** * Validates if generated compaction operation matches with input file slice and partition path. * - * @param slice File Slice - * @param op HoodieCompactionOperation + * @param slice File Slice + * @param op HoodieCompactionOperation * @param expPartitionPath Partition path */ private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompactionOperation op, String expPartitionPath, - int version) { + int version) { assertEquals(expPartitionPath, op.getPartitionPath(), "Partition path is correct"); assertEquals(slice.getBaseInstantTime(), op.getBaseInstantTime(), "Same base-instant"); assertEquals(slice.getFileId(), op.getFileId(), "Same file-id"); @@ -270,4 +362,24 @@ private void testFileSliceCompactionOpEquality(FileSlice slice, HoodieCompaction protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; } + + class MockHoodieActiveTimeline extends HoodieActiveTimeline { + + public MockHoodieActiveTimeline() { + super(); + this.setInstants(new ArrayList<>()); + } + + public MockHoodieActiveTimeline( + Stream completedDeltaCommits, + Stream completedCompactionCommits, + Stream> inflights) { + super(); + this.setInstants(Stream.concat( + Stream.concat(completedDeltaCommits.map(s -> new HoodieInstant(false, DELTA_COMMIT_ACTION, s)), + completedCompactionCommits.map(s -> new HoodieInstant(false, COMMIT_ACTION, s))), + inflights.map(s -> new HoodieInstant(true, s.getRight(), s.getLeft()))) + .sorted(Comparator.comparing(HoodieInstant::getFileName)).collect(Collectors.toList())); + } + } } From bf191f8d4662fd1e9c28f3cacce9179dc8cab561 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=86=AF=E5=81=A5?= Date: Thu, 17 Mar 2022 16:50:28 +0800 Subject: [PATCH 19/33] [HUDI-3645] Fix NPE caused by multiple threads accessing non-thread-safe HashMap (#5028) - Change HashMap in HoodieROTablePathFilter to ConcurrentHashMap --- .../java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 32dfd7c1e4ec2..be868ad29b284 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 @@ -45,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; /** @@ -93,7 +94,7 @@ public HoodieROTablePathFilter() { } public HoodieROTablePathFilter(Configuration conf) { - this.hoodiePathCache = new HashMap<>(); + this.hoodiePathCache = new ConcurrentHashMap<>(); this.nonHoodiePathCache = new HashSet<>(); this.conf = new SerializableConfiguration(conf); this.metaClientCache = new HashMap<>(); From 7446ff95a7b099d8016b7ca48fdeed9b290d710f Mon Sep 17 00:00:00 2001 From: Raymond Xu <2701446+xushiyan@users.noreply.github.com> Date: Thu, 17 Mar 2022 19:17:56 +0800 Subject: [PATCH 20/33] [HUDI-2439] Replace RDD with HoodieData in HoodieSparkTable and commit executors (#4856) - Adopt HoodieData in Spark action commit executors - Make Spark independent DeleteHelper, WriteHelper, MergeHelper in hudi-client-common - Make HoodieTable in WriteClient APIs have raw type to decouple with Client's generic types --- .../hudi/client/BaseHoodieWriteClient.java | 41 ++-- .../org/apache/hudi/table/HoodieTable.java | 4 +- .../cluster/strategy/UpdateStrategy.java | 2 +- .../action/commit/BaseBulkInsertHelper.java | 4 +- .../action/commit/HoodieDeleteHelper.java} | 68 +++--- .../action/commit/HoodieMergeHelper.java} | 21 +- .../action/commit/HoodieWriteHelper.java} | 40 ++-- .../hudi/client/HoodieFlinkWriteClient.java | 24 ++- .../table/HoodieFlinkCopyOnWriteTable.java | 5 +- .../hudi/client/HoodieJavaWriteClient.java | 21 +- .../table/HoodieJavaCopyOnWriteTable.java | 4 +- .../table/HoodieJavaMergeOnReadTable.java | 2 +- .../JavaBulkInsertCommitActionExecutor.java | 6 +- .../action/commit/JavaBulkInsertHelper.java | 5 +- ...BulkInsertPreppedCommitActionExecutor.java | 4 +- .../apache/hudi/client/HoodieReadClient.java | 2 +- .../hudi/client/SparkRDDWriteClient.java | 111 +++++----- .../MultipleSparkJobExecutionStrategy.java | 33 +-- .../SparkSingleFileSortExecutionStrategy.java | 6 +- .../SparkSortAndSizeExecutionStrategy.java | 6 +- .../strategy/BaseSparkUpdateStrategy.java | 9 +- .../strategy/SparkAllowUpdateStrategy.java | 5 +- .../strategy/SparkRejectUpdateStrategy.java | 4 +- .../client/utils/SparkValidatorUtils.java | 24 +-- .../validator/SparkPreCommitValidator.java | 7 +- .../SqlQueryEqualityPreCommitValidator.java | 4 +- .../SqlQueryInequalityPreCommitValidator.java | 4 +- .../validator/SqlQueryPreCommitValidator.java | 5 +- ...qlQuerySingleResultPreCommitValidator.java | 4 +- .../apache/hudi/data/HoodieJavaPairRDD.java | 6 + .../org/apache/hudi/data/HoodieJavaRDD.java | 22 ++ .../table/HoodieSparkCopyOnWriteTable.java | 56 ++--- .../table/HoodieSparkMergeOnReadTable.java | 54 +++-- .../apache/hudi/table/HoodieSparkTable.java | 13 +- .../SparkBootstrapCommitActionExecutor.java | 53 +++-- ...arkBootstrapDeltaCommitActionExecutor.java | 8 +- ...ExecuteClusteringCommitActionExecutor.java | 9 +- .../commit/BaseSparkCommitActionExecutor.java | 105 +++++----- .../SparkBulkInsertCommitActionExecutor.java | 17 +- .../action/commit/SparkBulkInsertHelper.java | 59 +++--- ...BulkInsertPreppedCommitActionExecutor.java | 13 +- .../SparkDeleteCommitActionExecutor.java | 11 +- ...rkDeletePartitionCommitActionExecutor.java | 19 +- .../SparkInsertCommitActionExecutor.java | 11 +- ...rkInsertOverwriteCommitActionExecutor.java | 25 ++- ...ertOverwriteTableCommitActionExecutor.java | 26 +-- ...parkInsertPreppedCommitActionExecutor.java | 9 +- .../SparkUpsertCommitActionExecutor.java | 11 +- ...parkUpsertPreppedCommitActionExecutor.java | 9 +- .../HoodieSparkMergeOnReadTableCompactor.java | 8 +- .../BaseSparkDeltaCommitActionExecutor.java | 8 +- ...rkBulkInsertDeltaCommitActionExecutor.java | 17 +- ...nsertPreppedDeltaCommitActionExecutor.java | 13 +- .../SparkDeleteDeltaCommitActionExecutor.java | 13 +- .../SparkInsertDeltaCommitActionExecutor.java | 13 +- ...nsertPreppedDeltaCommitActionExecutor.java | 9 +- .../SparkUpsertDeltaCommitActionExecutor.java | 14 +- ...psertPreppedDeltaCommitActionExecutor.java | 9 +- .../TestHoodieClientOnCopyOnWriteStorage.java | 30 ++- .../table/TestHoodieMergeOnReadTable.java | 3 +- .../commit/TestCopyOnWriteActionExecutor.java | 18 +- .../table/action/commit/TestDeleteHelper.java | 194 ------------------ .../action/compact/TestHoodieCompactor.java | 5 +- .../apache/hudi/common/data/HoodieData.java | 8 + .../apache/hudi/common/data/HoodieList.java | 26 +++ .../hudi/common/data/HoodieMapPair.java | 11 + .../hudi/common/data/HoodiePairData.java | 3 + .../hudi/common/data/TestHoodieList.java | 50 +++++ .../hudi/common/data/TestHoodieMapPair.java | 29 +++ 69 files changed, 723 insertions(+), 769 deletions(-) rename hudi-client/{hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java => hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java} (53%) rename hudi-client/{hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java => hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java} (81%) rename hudi-client/{hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java => hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java} (60%) delete mode 100644 hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 9bad2e3486e7f..a6a7e18b1f6ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.async.AsyncArchiveService; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -78,6 +76,9 @@ import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; import org.apache.hudi.table.upgrade.UpgradeDowngrade; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -242,11 +243,11 @@ protected void commit(HoodieTable table, String commitActionType, String instant Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } - protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { return createTable(config, hadoopConf, false); } - protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { try { @@ -397,7 +398,7 @@ protected void rollbackFailedBootstrap() { * @return Collection of WriteStatus to inspect errors and counts */ public abstract O bulkInsert(I records, final String instantTime, - Option> userDefinedBulkInsertPartitioner); + Option userDefinedBulkInsertPartitioner); /** @@ -417,7 +418,7 @@ public abstract O bulkInsert(I records, final String instantTime, * @return Collection of WriteStatus to inspect errors and counts */ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime, - Option> bulkInsertPartitioner); + Option bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be @@ -458,7 +459,7 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp * @param hoodieTable Hoodie Table * @return Write Status */ - protected abstract O postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable); + protected abstract O postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable); /** * Post Commit Hook. Derived classes use this method to perform post-commit processing @@ -468,7 +469,7 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp * @param instantTime Instant Time * @param extraMetadata Additional Metadata passed by user */ - protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) @@ -480,7 +481,7 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me } } - protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { + protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { if (!tableServicesEnabled(config)) { return; } @@ -524,7 +525,7 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommi } } - protected void runAnyPendingCompactions(HoodieTable table) { + protected void runAnyPendingCompactions(HoodieTable table) { table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { LOG.info("Running previously failed inflight compaction at instant " + instant); @@ -532,7 +533,7 @@ protected void runAnyPendingCompactions(HoodieTable table) { }); } - protected void runAnyPendingClustering(HoodieTable table) { + protected void runAnyPendingClustering(HoodieTable table) { table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> { Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); if (instantPlan.isPresent()) { @@ -558,7 +559,7 @@ protected void autoCleanOnCommit() { } } - protected void autoArchiveOnCommit(HoodieTable table) { + protected void autoArchiveOnCommit(HoodieTable table) { if (!config.isAutoArchive()) { return; } @@ -808,7 +809,7 @@ public HoodieCleanMetadata clean(boolean skipLocking) { * and keep increasing unbounded over time. * @param table table to commit on. */ - protected void archive(HoodieTable table) { + protected void archive(HoodieTable table) { if (!tableServicesEnabled(config)) { return; } @@ -937,7 +938,7 @@ public abstract void commitCompaction(String compactionInstantTime, HoodieCommit /** * Commit Compaction and track metrics. */ - protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime); + protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime); /** * Get inflight time line exclude compaction and clustering. @@ -1223,7 +1224,7 @@ protected Option inlineScheduleClustering(Option> ex return scheduleClustering(extraMetadata); } - protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { Option pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false); String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); @@ -1238,7 +1239,7 @@ protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieT * @param instantTime Instant Time * @param stats Hoodie Write Stat */ - protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { + protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { try { final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); table.finalizeWrite(context, instantTime, stats); @@ -1273,7 +1274,7 @@ public HoodieMetrics getMetrics() { * @param instantTime current inflight instant time * @return instantiated {@link HoodieTable} */ - protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime); + protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime); /** * Instantiates and initializes instance of {@link HoodieTable}, performing crucial bootstrapping @@ -1288,14 +1289,14 @@ public HoodieMetrics getMetrics() { *
  • Initializing metrics contexts
  • * */ - protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { + protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); // Setup write schemas for deletes if (operationType == WriteOperationType.DELETE) { setWriteSchemaForDeletes(metaClient); } - HoodieTable table; + HoodieTable table; this.txnManager.beginTransaction(); try { @@ -1381,7 +1382,7 @@ public void close() { this.txnManager.close(); } - private void setWriteTimer(HoodieTable table) { + private void setWriteTimer(HoodieTable table) { String commitType = table.getMetaClient().getCommitActionType(); if (commitType.equals(HoodieTimeline.COMMIT_ACTION)) { writeTimer = metrics.getCommitCtx(); 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 e5262ad6bb9f2..62a4f089a45b5 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 @@ -180,7 +180,7 @@ public abstract HoodieWriteMetadata insert(HoodieEngineContext context, Strin * @return HoodieWriteMetadata */ public abstract HoodieWriteMetadata bulkInsert(HoodieEngineContext context, String instantTime, - I records, Option> bulkInsertPartitioner); + I records, Option bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be @@ -237,7 +237,7 @@ public abstract HoodieWriteMetadata insertPrepped(HoodieEngineContext context * @return HoodieWriteMetadata */ public abstract HoodieWriteMetadata bulkInsertPrepped(HoodieEngineContext context, String instantTime, - I preppedRecords, Option> bulkInsertPartitioner); + I preppedRecords, Option bulkInsertPartitioner); /** * Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime, 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 index 009790812a624..4e33eb06038cd 100644 --- 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 @@ -28,7 +28,7 @@ /** * When file groups in clustering, write records to these file group need to check. */ -public abstract class UpdateStrategy, I> { +public abstract class UpdateStrategy { protected final HoodieEngineContext engineContext; protected Set fileGroupsInPendingClustering; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java index dffd926aee3d5..ad2145c3501bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java @@ -34,7 +34,7 @@ public abstract class BaseBulkInsertHelper bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, BaseCommitActionExecutor executor, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner); + Option userDefinedBulkInsertPartitioner); /** * Only write input records. Does not change timeline/index. Return information about new files created. @@ -42,7 +42,7 @@ public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instant public abstract O bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner, + Option userDefinedBulkInsertPartitioner, boolean addMetadataFields, int parallelism, WriteHandleFactory writeHandleFactory); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java similarity index 53% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java index 381c115533897..fff52eb24d736 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java @@ -7,19 +7,20 @@ * "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 + * 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. + * 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.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; @@ -28,16 +29,12 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; 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 org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import java.time.Duration; import java.time.Instant; import java.util.HashMap; @@ -48,69 +45,64 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkDeleteHelper extends - BaseDeleteHelper>, JavaRDD, JavaRDD, R> { - private SparkDeleteHelper() { +public class HoodieDeleteHelper extends + BaseDeleteHelper>, HoodieData, HoodieData, R> { + private HoodieDeleteHelper() { } private static class DeleteHelperHolder { - private static final SparkDeleteHelper SPARK_DELETE_HELPER = new SparkDeleteHelper(); + private static final HoodieDeleteHelper HOODIE_DELETE_HELPER = new HoodieDeleteHelper<>(); } - public static SparkDeleteHelper newInstance() { - return DeleteHelperHolder.SPARK_DELETE_HELPER; + public static HoodieDeleteHelper newInstance() { + return DeleteHelperHolder.HOODIE_DELETE_HELPER; } @Override - public JavaRDD deduplicateKeys(JavaRDD keys, HoodieTable>, JavaRDD, JavaRDD> table, int parallelism) { + public HoodieData deduplicateKeys(HoodieData keys, HoodieTable>, HoodieData, HoodieData> table, int parallelism) { boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { - return keys.keyBy(HoodieKey::getRecordKey) - .reduceByKey((key1, key2) -> key1, parallelism) - .values(); + return keys.distinctWithKey(HoodieKey::getRecordKey, parallelism); } else { return keys.distinct(parallelism); } } @Override - public HoodieWriteMetadata> execute(String instantTime, - JavaRDD keys, - HoodieEngineContext context, - HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, - BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> deleteExecutor) { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); - + public HoodieWriteMetadata> execute(String instantTime, + HoodieData keys, + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + BaseCommitActionExecutor>, HoodieData, HoodieData, R> deleteExecutor) { try { - HoodieWriteMetadata result = null; - JavaRDD dedupedKeys = keys; + HoodieData dedupedKeys = keys; final int parallelism = config.getDeleteShuffleParallelism(); if (config.shouldCombineBeforeDelete()) { // De-dupe/merge if needed dedupedKeys = deduplicateKeys(keys, table, parallelism); - } else if (!keys.partitions().isEmpty()) { + } else if (!keys.isEmpty()) { dedupedKeys = keys.repartition(parallelism); } - JavaRDD> dedupedRecords = + HoodieData> dedupedRecords = dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); Instant beginTag = Instant.now(); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = HoodieJavaRDD.getJavaRDD( - table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); + HoodieData> taggedRecords = table.getIndex().tagLocation(dedupedRecords, context, table); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records - JavaRDD> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown); + HoodieData> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown); + HoodieWriteMetadata> result; 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(jsc.emptyRDD()); + result = new HoodieWriteMetadata<>(); + result.setWriteStatuses(context.emptyHoodieData()); deleteExecutor.commitOnAutoCommit(result); } return result; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java similarity index 81% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index e87c3ef5ba77e..2b4a5d1608eec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -37,31 +38,29 @@ import org.apache.avro.io.BinaryDecoder; import org.apache.avro.io.BinaryEncoder; import org.apache.hadoop.conf.Configuration; -import org.apache.spark.api.java.JavaRDD; import java.io.IOException; import java.util.Iterator; -public class SparkMergeHelper extends BaseMergeHelper>, - JavaRDD, JavaRDD> { +public class HoodieMergeHelper extends + BaseMergeHelper>, HoodieData, HoodieData> { - private SparkMergeHelper() { + private HoodieMergeHelper() { } private static class MergeHelperHolder { - private static final SparkMergeHelper SPARK_MERGE_HELPER = new SparkMergeHelper(); + private static final HoodieMergeHelper HOODIE_MERGE_HELPER = new HoodieMergeHelper<>(); } - public static SparkMergeHelper newInstance() { - return SparkMergeHelper.MergeHelperHolder.SPARK_MERGE_HELPER; + public static HoodieMergeHelper newInstance() { + return MergeHelperHolder.HOODIE_MERGE_HELPER; } @Override - public void runMerge(HoodieTable>, JavaRDD, JavaRDD> table, - HoodieMergeHandle>, JavaRDD, JavaRDD> upsertHandle) throws IOException { + public void runMerge(HoodieTable>, HoodieData, HoodieData> table, + HoodieMergeHandle>, HoodieData, HoodieData> mergeHandle) throws IOException { final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation(); Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf()); - HoodieMergeHandle>, JavaRDD, JavaRDD> mergeHandle = upsertHandle; HoodieBaseFile baseFile = mergeHandle.baseFileForMerge(); final GenericDatumWriter gWriter; @@ -78,7 +77,7 @@ public void runMerge(HoodieTable>, JavaRDD } BoundedInMemoryExecutor wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); try { final Iterator readerIterator; if (baseFile.getBootstrapBaseFile().isPresent()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java similarity index 60% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java index 23dceb1382f34..b56d39b8e3679 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java @@ -19,60 +19,52 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroRecord; 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.data.HoodieJavaRDD; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; +public class HoodieWriteHelper extends BaseWriteHelper>, + HoodieData, HoodieData, R> { -import scala.Tuple2; - -/** - * A spark implementation of {@link BaseWriteHelper}. - * - * @param - */ -public class SparkWriteHelper extends BaseWriteHelper>, - JavaRDD, JavaRDD, R> { - private SparkWriteHelper() { + private HoodieWriteHelper() { } private static class WriteHelperHolder { - private static final SparkWriteHelper SPARK_WRITE_HELPER = new SparkWriteHelper(); + private static final HoodieWriteHelper HOODIE_WRITE_HELPER = new HoodieWriteHelper<>(); } - public static SparkWriteHelper newInstance() { - return WriteHelperHolder.SPARK_WRITE_HELPER; + public static HoodieWriteHelper newInstance() { + return WriteHelperHolder.HOODIE_WRITE_HELPER; } @Override - protected JavaRDD> tag(JavaRDD> dedupedRecords, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> table) { - return HoodieJavaRDD.getJavaRDD( - table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); + protected HoodieData> tag(HoodieData> dedupedRecords, HoodieEngineContext context, + HoodieTable>, HoodieData, HoodieData> table) { + return table.getIndex().tagLocation(dedupedRecords, context, table); } @Override - public JavaRDD> deduplicateRecords( - JavaRDD> records, HoodieIndex index, int parallelism) { + public HoodieData> deduplicateRecords( + HoodieData> records, HoodieIndex index, int parallelism) { boolean isIndexingGlobal = index.isGlobal(); return records.mapToPair(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 new Tuple2<>(key, record); + return Pair.of(key, record); }).reduceByKey((rec1, rec2) -> { @SuppressWarnings("unchecked") T reducedData = (T) rec2.getData().preCombine(rec1.getData()); HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); - return new HoodieAvroRecord(reducedKey, reducedData); - }, parallelism).map(Tuple2::_2); + return new HoodieAvroRecord<>(reducedKey, reducedData); + }, parallelism).map(Pair::getRight); } } 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 fb613309d3fa2..4523705f32b74 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,8 +18,6 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.data.HoodieList; @@ -55,6 +53,7 @@ import org.apache.hudi.io.MiniBatchHandle; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -64,6 +63,9 @@ import org.apache.hudi.table.upgrade.FlinkUpgradeDowngradeHelper; import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.hudi.util.FlinkClientUtil; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,8 +113,8 @@ public boolean commit(String instantTime, List writeStatuses, Optio } @Override - protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, + boolean refreshTimeline) { return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); } @@ -226,12 +228,12 @@ public List bulkInsert(List> records, String instan } @Override - public List bulkInsert(List> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { + public List bulkInsert(List> records, String instantTime, Option userDefinedBulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert operation is not supported yet"); } @Override - public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { + public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsertPrepped operation is not supported yet"); } @@ -304,7 +306,7 @@ public void waitForCleaningFinish() { @Override protected List postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, List, List> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -324,7 +326,7 @@ protected List postWrite(HoodieWriteMetadata> res * @param extraMetadata Additional Metadata passed by user */ @Override - protected void postCommit(HoodieTable>, List, List> table, + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { @@ -351,7 +353,7 @@ public void commitCompaction( @Override public void completeCompaction( HoodieCommitMetadata metadata, - HoodieTable>, List, List> table, + HoodieTable table, String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = metadata.getWriteStats(); @@ -363,7 +365,7 @@ public void completeCompaction( // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent( - w -> w.update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); + w -> ((HoodieTableMetadataWriter) w).update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); } finally { @@ -396,7 +398,7 @@ public HoodieWriteMetadata> cluster(final String clusteringIns } @Override - protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // Create a Hoodie table which encapsulated the commits and files visible return getHoodieTable(); } 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 7e41ab150fbf2..14937d6fee244 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 @@ -68,6 +68,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; + import java.io.IOException; import java.util.Collections; import java.util.Iterator; @@ -231,7 +232,7 @@ public HoodieWriteMetadata> insert(HoodieEngineContext context public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, List> records, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert is not supported yet"); } @@ -264,7 +265,7 @@ public HoodieWriteMetadata> insertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsertPrepped is not supported yet"); } 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 index 9de9298c25ae9..faf46e08d5515 100644 --- 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 @@ -18,8 +18,6 @@ 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.data.HoodieList; @@ -43,6 +41,9 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.upgrade.JavaUpgradeDowngradeHelper; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; + import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -88,9 +89,9 @@ public boolean commit(String instantTime, } @Override - protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, + Configuration hadoopConf, + boolean refreshTimeline) { return HoodieJavaTable.create(config, context); } @@ -152,7 +153,7 @@ public List bulkInsert(List> records, @Override public List bulkInsert(List> records, String instantTime, - Option>>> userDefinedBulkInsertPartitioner) { + Option userDefinedBulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert is not supported in HoodieJavaClient"); } @@ -166,7 +167,7 @@ public void transitionInflight(String instantTime) { @Override public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { HoodieTable>, List, List> table = initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); @@ -188,7 +189,7 @@ public List delete(List keys, @Override protected List postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, List, List> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -215,7 +216,7 @@ public void commitCompaction(String compactionInstantTime, @Override protected void completeCompaction(HoodieCommitMetadata metadata, - HoodieTable>, List, List> table, + HoodieTable table, String compactionCommitTime) { throw new HoodieNotSupportedException("CompleteCompaction is not supported in HoodieJavaClient"); } @@ -232,7 +233,7 @@ public HoodieWriteMetadata> cluster(final String clusteringIns } @Override - protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); // Create a Hoodie table which encapsulated the commits and files visible 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 index 447ed3e96cd9e..06c23049d974d 100644 --- 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 @@ -114,7 +114,7 @@ public HoodieWriteMetadata> insert(HoodieEngineContext context public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, List> records, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, records, bulkInsertPartitioner).execute(); } @@ -152,7 +152,7 @@ public HoodieWriteMetadata> insertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); } 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 index 136c25b8cdb2a..32d30f704ecbb 100644 --- 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 @@ -61,7 +61,7 @@ public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java index 9780262fb2b92..d5c7a0b0b5dc7 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java @@ -36,17 +36,17 @@ public class JavaBulkInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { private final List> inputRecords; - private final Option>>> bulkInsertPartitioner; + private final Option bulkInsertPartitioner; public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecords, bulkInsertPartitioner, Option.empty()); } public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecords = inputRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java index de7afdf00ebeb..30f1d931a5462 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java @@ -65,7 +65,7 @@ public HoodieWriteMetadata> bulkInsert(final List>, List, List, R> executor, final boolean performDedupe, - final Option>>> userDefinedBulkInsertPartitioner) { + final Option userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); // It's possible the transition to inflight could have already happened. @@ -89,7 +89,7 @@ public List bulkInsert(List> inputRecords, HoodieTable>, List, List> table, HoodieWriteConfig config, boolean performDedupe, - Option>>> userDefinedBulkInsertPartitioner, + Option userDefinedBulkInsertPartitioner, boolean useWriterSchema, int parallelism, WriteHandleFactory writeHandleFactory) { @@ -106,6 +106,7 @@ public List bulkInsert(List> inputRecords, BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : JavaBulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); + // only List is supported for Java partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463 repartitionedRecords = (List>) partitioner.repartitionRecords(dedupedRecords, parallelism); FileIdPrefixProvider fileIdPrefixProvider = (FileIdPrefixProvider) ReflectionUtils.loadClass( diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java index ed72fbe7850e2..14c4c8a93e916 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java @@ -36,12 +36,12 @@ public class JavaBulkInsertPreppedCommitActionExecutor { private final List> preppedInputRecord; - private final Option>>> userDefinedBulkInsertPartitioner; + private final Option userDefinedBulkInsertPartitioner; public JavaBulkInsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> preppedInputRecord, - Option>>> userDefinedBulkInsertPartitioner) { + Option userDefinedBulkInsertPartitioner) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecord = preppedInputRecord; this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java index e9bdc427e8356..37a78a4be54c0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java @@ -68,7 +68,7 @@ public class HoodieReadClient> implements Seria * base path pointing to the table. Until, then just always assume a BloomIndex */ private final transient HoodieIndex index; - private HoodieTable>, JavaRDD, JavaRDD> hoodieTable; + private HoodieTable hoodieTable; private transient Option sqlContextOpt; private final transient HoodieSparkEngineContext context; private final transient Configuration hadoopConf; 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 ec1ecd6a0c291..ac9259c51ad70 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,11 +18,10 @@ package org.apache.hudi.client; -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.client.utils.TransactionUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.metrics.Registry; @@ -45,6 +44,7 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndexFactory; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.table.BulkInsertPartitioner; @@ -54,6 +54,9 @@ import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper; + +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; @@ -120,9 +123,9 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op } @Override - protected HoodieTable>, JavaRDD, JavaRDD> createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, + Configuration hadoopConf, + boolean refreshTimeline) { return HoodieSparkTable.create(config, context, refreshTimeline); } @@ -147,45 +150,49 @@ public void bootstrap(Option> extraMetadata) { @Override public JavaRDD upsert(JavaRDD> records, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime)); table.validateUpsertSchema(); preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.upsert(context, instantTime, records); + HoodieWriteMetadata> result = table.upsert(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); } - return postWrite(result, instantTime, table); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.UPSERT_PREPPED, Option.ofNullable(instantTime)); table.validateUpsertSchema(); preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD insert(JavaRDD> records, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.insert(context,instantTime, records); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.insert(context,instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } /** @@ -196,11 +203,12 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { - HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient()); - HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.insertOverwrite(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } /** @@ -211,11 +219,12 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public HoodieWriteResult insertOverwriteTable(JavaRDD> records, final String instantTime) { - HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient()); - HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.insertOverwriteTable(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } @Override @@ -224,44 +233,48 @@ public JavaRDD bulkInsert(JavaRDD> records, String } @Override - public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { - HoodieTable>, JavaRDD, JavaRDD> table = + public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option userDefinedBulkInsertPartitioner) { + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.BULK_INSERT, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.BULK_INSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, HoodieJavaRDD.of(records), userDefinedBulkInsertPartitioner); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override - public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { - HoodieTable>, JavaRDD, JavaRDD> table = + public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option bulkInsertPartitioner) { + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords), bulkInsertPartitioner); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD delete(JavaRDD keys, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime)); preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); - HoodieWriteMetadata> result = table.delete(context,instantTime, keys); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.delete(context,instantTime, HoodieJavaRDD.of(keys)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } public HoodieWriteResult deletePartitions(List partitions, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE_PARTITION, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.DELETE_PARTITION, Option.ofNullable(instantTime)); preWrite(instantTime, WriteOperationType.DELETE_PARTITION, table.getMetaClient()); - HoodieWriteMetadata> result = table.deletePartitions(context, instantTime, partitions); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.deletePartitions(context, instantTime, partitions); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } @Override protected JavaRDD postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -288,7 +301,7 @@ public void commitCompaction(String compactionInstantTime, HoodieCommitMetadata @Override protected void completeCompaction(HoodieCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = metadata.getWriteStats(); @@ -329,8 +342,8 @@ protected HoodieWriteMetadata> compact(String compactionIns table.getMetaClient().reloadActiveTimeline(); } compactionTimer = metrics.getCompactionCtx(); - HoodieWriteMetadata> compactionMetadata = - table.compact(context, compactionInstantTime); + HoodieWriteMetadata> writeMetadata = table.compact(context, compactionInstantTime); + HoodieWriteMetadata> compactionMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) { completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), table, compactionInstantTime); } @@ -349,7 +362,8 @@ public HoodieWriteMetadata> cluster(String clusteringInstan } clusteringTimer = metrics.getClusteringCtx(); LOG.info("Starting clustering at " + clusteringInstant); - HoodieWriteMetadata> clusteringMetadata = table.cluster(context, clusteringInstant); + HoodieWriteMetadata> writeMetadata = table.cluster(context, clusteringInstant); + HoodieWriteMetadata> clusteringMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); // TODO : Where is shouldComplete used ? if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) { completeTableService(TableServiceType.CLUSTER, clusteringMetadata.getCommitMetadata().get(), table, clusteringInstant); @@ -358,9 +372,8 @@ public HoodieWriteMetadata> cluster(String clusteringInstan } private void completeClustering(HoodieReplaceCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, - String clusteringCommitTime) { - + HoodieTable table, + String clusteringCommitTime) { List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> e.getValue().stream()).collect(Collectors.toList()); @@ -405,17 +418,17 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, LOG.info("Clustering successfully on commit " + clusteringCommitTime); } - private void updateTableMetadata(HoodieTable>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata, + private void updateTableMetadata(HoodieTable table, HoodieCommitMetadata commitMetadata, HoodieInstant hoodieInstant) { boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction()); // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. table.getMetadataWriter(hoodieInstant.getTimestamp()) - .ifPresent(writer -> writer.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); + .ifPresent(writer -> ((HoodieTableMetadataWriter) writer).update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); } @Override - protected HoodieTable>, JavaRDD, JavaRDD> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation, // if it didn't exist before // See https://issues.apache.org/jira/browse/HUDI-3343 for more details @@ -440,7 +453,7 @@ private void initializeMetadataTable(Option inFlightInstantTimestamp) { // TODO : To enforce priority between table service and ingestion writer, use transactions here and invoke strategy private void completeTableService(TableServiceType tableServiceType, HoodieCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, String commitInstant) { switch (tableServiceType) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 54c1c9f5ac05d..5a03cdf3bc9a1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -18,10 +18,6 @@ package org.apache.hudi.client.clustering.run.strategy; -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; @@ -62,6 +58,11 @@ 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.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -94,7 +95,7 @@ public MultipleSparkJobExecutionStrategy(HoodieTable table, HoodieEngineContext public HoodieWriteMetadata> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) { JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext()); // execute clustering for each group async and collect WriteStatus - Stream> writeStatusRDDStream = FutureUtils.allOf( + Stream> writeStatusesStream = FutureUtils.allOf( clusteringPlan.getInputGroups().stream() .map(inputGroup -> runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), @@ -103,7 +104,7 @@ public HoodieWriteMetadata> performClustering(final Hood .collect(Collectors.toList())) .join() .stream(); - JavaRDD[] writeStatuses = convertStreamToArray(writeStatusRDDStream); + JavaRDD[] writeStatuses = convertStreamToArray(writeStatusesStream.map(HoodieJavaRDD::getJavaRDD)); JavaRDD writeStatusRDD = engineContext.union(writeStatuses); HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); @@ -125,7 +126,7 @@ public HoodieWriteMetadata> performClustering(final Hood * @param preserveHoodieMetadata Whether to preserve commit metadata while clustering. * @return RDD of {@link WriteStatus}. */ - public abstract JavaRDD performClusteringWithRecordsRDD(final JavaRDD> inputRecords, final int numOutputGroups, final String instantTime, + public abstract HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); @@ -164,11 +165,11 @@ protected Option>>> getPartitioner /** * Submit job to execute clustering for the group. */ - private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, + private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, boolean preserveHoodieMetadata, String instantTime) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); - JavaRDD> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); + HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) @@ -180,7 +181,7 @@ private CompletableFuture> runClusteringForGroupAsync(Hoodi /** * 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, String instantTime) { + private HoodieData> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); if (hasLogFiles) { @@ -195,12 +196,12 @@ private JavaRDD> readRecordsForGroup(JavaSparkContext jsc, Hoodi /** * Read records from baseFiles, apply updates and convert to RDD. */ - private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext jsc, + private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext jsc, List clusteringOps, String instantTime) { HoodieWriteConfig config = getWriteConfig(); HoodieTable table = getHoodieTable(); - return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { List>> recordIterators = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config); @@ -237,20 +238,20 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js }); return new ConcatenatingIterator<>(recordIterators); - }); + })); } /** * Read records from baseFiles and convert to RDD. */ - private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext jsc, + private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContext jsc, List clusteringOps) { SerializableConfiguration hadoopConf = new SerializableConfiguration(getHoodieTable().getHadoopConf()); HoodieWriteConfig writeConfig = getWriteConfig(); // NOTE: It's crucial to make sure that we don't capture whole "this" object into the // closure, as this might lead to issues attempting to serialize its nested fields - return jsc.parallelize(clusteringOps, clusteringOps.size()) + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()) .mapPartitions(clusteringOpsPartition -> { List> iteratorsForPartition = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { @@ -266,7 +267,7 @@ private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext j return new ConcatenatingIterator<>(iteratorsForPartition); }) - .map(record -> transform(record, writeConfig)); + .map(record -> transform(record, writeConfig))); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index 2a503931b496b..4dedabaec850e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -20,6 +20,7 @@ package org.apache.hudi.client.clustering.run.strategy; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -34,7 +35,6 @@ import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.Map; @@ -56,7 +56,7 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, } @Override - public JavaRDD performClusteringWithRecordsRDD(JavaRDD> inputRecords, + public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, int numOutputGroups, String instantTime, Map strategyParams, @@ -74,7 +74,7 @@ public JavaRDD performClusteringWithRecordsRDD(JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + return (HoodieData) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), preserveHoodieMetadata)); } } 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 index 22d5300215d0b..d664c835e494b 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.hudi.client.clustering.run.strategy; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -32,7 +33,6 @@ import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.Map; @@ -54,7 +54,7 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, } @Override - public JavaRDD performClusteringWithRecordsRDD(final JavaRDD> inputRecords, final int numOutputGroups, + public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); @@ -64,7 +64,7 @@ public JavaRDD performClusteringWithRecordsRDD(final JavaRDD) SparkBulkInsertHelper.newInstance() + return (HoodieData) SparkBulkInsertHelper.newInstance() .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java index 655c11983e46b..3eadba25bbcfb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java @@ -19,13 +19,12 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.table.action.cluster.strategy.UpdateStrategy; -import org.apache.spark.api.java.JavaRDD; - import java.util.List; import java.util.Set; @@ -33,7 +32,7 @@ * Spark base update strategy, write records to the file groups which are in clustering * need to check. Spark relate implementations should extend this base class. */ -public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { +public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { public BaseSparkUpdateStrategy(HoodieSparkEngineContext engineContext, Set fileGroupsInPendingClustering) { @@ -45,9 +44,9 @@ public BaseSparkUpdateStrategy(HoodieSparkEngineContext engineContext, * @param inputRecords the records to write, tagged with target file id * @return the records matched file group ids */ - protected List getGroupIdsWithUpdate(JavaRDD> inputRecords) { + protected List getGroupIdsWithUpdate(HoodieData> inputRecords) { return inputRecords .filter(record -> record.getCurrentLocation() != null) - .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collect(); + .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collectAsList(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java index 92a5fb69a7cd9..59040629f718e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java @@ -19,13 +19,12 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.common.util.collection.Pair; -import org.apache.spark.api.java.JavaRDD; - import java.util.HashSet; import java.util.List; import java.util.Set; @@ -42,7 +41,7 @@ public SparkAllowUpdateStrategy(HoodieSparkEngineContext engineContext, } @Override - public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { + public Pair>, Set> handleUpdate(HoodieData> taggedRecordsRDD) { List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); Set fileGroupIdsWithUpdatesAndPendingClustering = fileGroupIdsWithRecordUpdate.stream() .filter(f -> fileGroupsInPendingClustering.contains(f)) 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 index ac058a4d85095..d09422ee77242 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -27,7 +28,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.Collections; import java.util.HashSet; @@ -47,7 +47,7 @@ public SparkRejectUpdateStrategy(HoodieSparkEngineContext engineContext, } @Override - public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { + public Pair>, Set> handleUpdate(HoodieData> taggedRecordsRDD) { List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); fileGroupIdsWithRecordUpdate.forEach(fileGroupIdWithRecordUpdate -> { if (fileGroupsInPendingClustering.contains(fileGroupIdWithRecordUpdate)) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java index 9e72390e49f55..fd083f2c89a46 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java @@ -21,7 +21,9 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.validator.SparkPreCommitValidator; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.table.view.HoodieTablePreCommitFileSystemView; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -34,13 +36,11 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; import java.util.Arrays; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -61,7 +61,7 @@ public class SparkValidatorUtils { * Throw error if there are validation failures. */ public static void runValidators(HoodieWriteConfig config, - HoodieWriteMetadata> writeMetadata, + HoodieWriteMetadata> writeMetadata, HoodieEngineContext context, HoodieTable table, String instantTime) { @@ -69,10 +69,10 @@ public static void runValidators(HoodieWriteConfig config, LOG.info("no validators configured."); } else { if (!writeMetadata.getWriteStats().isPresent()) { - writeMetadata.setWriteStats(writeMetadata.getWriteStatuses().map(WriteStatus::getStat).collect()); + writeMetadata.setWriteStats(writeMetadata.getWriteStatuses().map(WriteStatus::getStat).collectAsList()); } - Set partitionsModified = new HashSet<>(writeMetadata.getWriteStats().get().stream().map(writeStats -> - writeStats.getPartitionPath()).collect(Collectors.toList())); + Set partitionsModified = writeMetadata.getWriteStats().get().stream().map(writeStats -> + writeStats.getPartitionPath()).collect(Collectors.toSet()); SQLContext sqlContext = new SQLContext(HoodieSparkEngineContext.getSparkContext(context)); // Refresh timeline to ensure validator sees the any other operations done on timeline (async operations such as other clustering/compaction/rollback) table.getMetaClient().reloadActiveTimeline(); @@ -101,8 +101,8 @@ public static void runValidators(HoodieWriteConfig config, /** * Run validators in a separate thread pool for parallelism. Each of validator can submit a distributed spark job if needed. */ - private static CompletableFuture runValidatorAsync(SparkPreCommitValidator validator, HoodieWriteMetadata writeMetadata, - Dataset beforeState, Dataset afterState, String instantTime) { + private static CompletableFuture runValidatorAsync(SparkPreCommitValidator validator, HoodieWriteMetadata> writeMetadata, + Dataset beforeState, Dataset afterState, String instantTime) { return CompletableFuture.supplyAsync(() -> { try { validator.validate(instantTime, writeMetadata, beforeState, afterState); @@ -120,10 +120,10 @@ private static CompletableFuture runValidatorAsync(SparkPreCommitValida * Note that this only works for COW tables. */ public static Dataset getRecordsFromCommittedFiles(SQLContext sqlContext, - Set partitionsAffected, HoodieTable table) { + Set partitionsAffected, HoodieTable table) { List committedFiles = partitionsAffected.stream() - .flatMap(partition -> table.getBaseFileOnlyView().getLatestBaseFiles(partition).map(bf -> bf.getPath())) + .flatMap(partition -> table.getBaseFileOnlyView().getLatestBaseFiles(partition).map(BaseFile::getPath)) .collect(Collectors.toList()); if (committedFiles.isEmpty()) { @@ -145,7 +145,7 @@ public static Dataset readRecordsForBaseFiles(SQLContext sqlContext, List getRecordsFromPendingCommits(SQLContext sqlContext, Set partitionsAffected, - HoodieWriteMetadata> writeMetadata, + HoodieWriteMetadata> writeMetadata, HoodieTable table, String instantTime) { @@ -157,7 +157,7 @@ public static Dataset getRecordsFromPendingCommits(SQLContext sqlContext, instantTime); List newFiles = partitionsAffected.stream() - .flatMap(partition -> fsView.getLatestBaseFiles(partition).map(bf -> bf.getPath())) + .flatMap(partition -> fsView.getLatestBaseFiles(partition).map(BaseFile::getPath)) .collect(Collectors.toList()); if (newFiles.isEmpty()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java index f12d337bb12e9..f08d11b571492 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; @@ -28,9 +29,9 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -41,7 +42,7 @@ /** * Validator can be configured pre-commit. */ -public abstract class SparkPreCommitValidator> { +public abstract class SparkPreCommitValidator> { private static final Logger LOG = LogManager.getLogger(SparkPreCommitValidator.class); private HoodieSparkTable table; @@ -59,7 +60,7 @@ protected Set getPartitionsModified(HoodieWriteMetadata writeResult) if (writeResult.getWriteStats().isPresent()) { partitionsModified = writeResult.getWriteStats().get().stream().map(HoodieWriteStat::getPartitionPath).collect(Collectors.toSet()); } else { - partitionsModified = new HashSet<>(writeResult.getWriteStatuses().map(WriteStatus::getPartitionPath).collect()); + partitionsModified = new HashSet<>(writeResult.getWriteStatuses().map(WriteStatus::getPartitionPath).collectAsList()); } return partitionsModified; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java index b27f84e41c28c..2506d52b4a416 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ * * Expects both queries to return same result. */ -public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryEqualityPreCommitValidator.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java index 026334fde0cde..8a25150651943 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ *

    * Expects query results do not match. */ -public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQueryInequalityPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java index 122cf2be5c841..3a88d54d36d7b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java @@ -20,15 +20,16 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.table.HoodieSparkTable; + 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.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -41,7 +42,7 @@ /** * Validator framework to run sql queries and compare table state at different locations. */ -public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { +public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryPreCommitValidator.class); private static final AtomicInteger TABLE_COUNTER = new AtomicInteger(0); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java index 66e956dc59650..b1942244d3d39 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ *

    * Example configuration: "query1#expectedResult1;query2#expectedResult2;" */ -public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQuerySingleResultPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java index ffa1a35652c3c..ddcaaec0fa6ca 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -103,6 +104,11 @@ public Map countByKey() { return pairRDDData.countByKey(); } + @Override + public HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism) { + return HoodieJavaPairRDD.of(pairRDDData.reduceByKey(func::apply, parallelism)); + } + @Override public HoodieData map(SerializableFunction, O> func) { return HoodieJavaRDD.of(pairRDDData.map( diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java index 1381ea86e481c..66edf607f84dd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java @@ -130,6 +130,23 @@ public HoodieData distinct() { return HoodieJavaRDD.of(rddData.distinct()); } + @Override + public HoodieData distinct(int parallelism) { + return HoodieJavaRDD.of(rddData.distinct(parallelism)); + } + + @Override + public HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism) { + return mapToPair(i -> Pair.of(keyGetter.apply(i), i)) + .reduceByKey((value1, value2) -> value1, parallelism) + .values(); + } + + @Override + public HoodieData filter(SerializableFunction filterFunc) { + return HoodieJavaRDD.of(rddData.filter(filterFunc::apply)); + } + @Override public HoodieData union(HoodieData other) { return HoodieJavaRDD.of(rddData.union((JavaRDD) other.get())); @@ -139,4 +156,9 @@ public HoodieData union(HoodieData other) { public List collectAsList() { return rddData.collect(); } + + @Override + public HoodieData repartition(int parallelism) { + return HoodieJavaRDD.of(rddData.repartition(parallelism)); + } } 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 31bd436612c11..8f5211212253a 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 @@ -32,6 +32,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -61,6 +62,7 @@ import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; +import org.apache.hudi.table.action.commit.HoodieMergeHelper; import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; @@ -69,7 +71,6 @@ 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; import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; @@ -82,7 +83,6 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import javax.annotation.Nonnull; @@ -117,58 +117,58 @@ public boolean isTableServiceAction(String actionType) { } @Override - public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertCommitActionExecutor<>((HoodieSparkEngineContext)context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, - Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, config, + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, HoodieData> records, + Option userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, HoodieData keys) { 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(); + 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) { + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords, Option userDefinedBulkInsertPartitioner) { return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute(); } @@ -235,7 +235,7 @@ public Option scheduleCompaction(HoodieEngineContext conte } @Override - public HoodieWriteMetadata> compact( + public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } @@ -248,20 +248,20 @@ public Option scheduleClustering(HoodieEngineContext conte } @Override - public HoodieWriteMetadata> cluster(HoodieEngineContext context, + public HoodieWriteMetadata> cluster(HoodieEngineContext context, String clusteringInstantTime) { return new SparkExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); } @Override - public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); } @Override public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { new RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - new CopyOnWriteRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + new CopyOnWriteRestoreActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); } @Override @@ -292,7 +292,7 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle> handleInsert( @Override public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking) { - return new CleanActionExecutor(context, config, this, cleanInstantTime, skipLocking).execute(); + return new CleanActionExecutor<>(context, config, this, cleanInstantTime, skipLocking).execute(); } @Override public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants, boolean skipLocking) { - return new CopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, + return new CopyOnWriteRollbackActionExecutor<>(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } @Override public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) { - return new SavepointActionExecutor(context, config, this, instantToSavepoint, user, comment).execute(); + return new SavepointActionExecutor<>(context, config, this, instantToSavepoint, user, comment).execute(); } @Override public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new CopyOnWriteRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new CopyOnWriteRestoreActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } @Override public Option scheduleRestore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new RestorePlanActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new RestorePlanActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } } 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 334efa7fc91f4..efc667af297be 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 @@ -24,6 +24,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -31,14 +32,13 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; - import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; +import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor; import org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor; import org.apache.hudi.table.action.compact.RunCompactionActionExecutor; import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor; @@ -54,8 +54,6 @@ import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; import org.apache.hudi.table.action.rollback.RestorePlanActionExecutor; -import org.apache.spark.api.java.JavaRDD; - import java.util.List; import java.util.Map; @@ -87,72 +85,72 @@ public boolean isTableServiceAction(String actionType) { } @Override - public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD> records, - Option>>> userDefinedBulkInsertPartitioner) { - return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, HoodieData> records, + Option userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, HoodieData keys) { return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); } @Override - public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords, Option userDefinedBulkInsertPartitioner) { return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { - ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( + ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor<>( context, config, this, instantTime, extraMetadata, - new HoodieSparkMergeOnReadTableCompactor()); + new HoodieSparkMergeOnReadTableCompactor<>()); return scheduleCompactionExecutor.execute(); } @Override - public HoodieWriteMetadata> compact( + public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { - RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor( - context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor(), - new HoodieSparkCopyOnWriteTable(config, context, getMetaClient())); - return convertMetadata(compactionExecutor.execute()); + RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor<>( + context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(), + new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient())); + return compactionExecutor.execute(); } @Override - public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { - return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { + return new SparkBootstrapDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); } @Override public void rollbackBootstrap(HoodieEngineContext context, String instantTime) { new RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - new MergeOnReadRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + new MergeOnReadRestoreActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); } @Override @@ -169,12 +167,12 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, HoodieInstant commitInstant, boolean deleteInstants, boolean skipLocking) { - return new MergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); + return new MergeOnReadRollbackActionExecutor<>(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } @Override public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new MergeOnReadRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new MergeOnReadRestoreActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } @Override 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 bb8c95d745ab1..ce14d43cfc6e9 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 @@ -36,20 +36,16 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; -import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.fs.Path; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; -import org.apache.spark.api.java.JavaRDD; import java.io.IOException; -import static org.apache.hudi.data.HoodieJavaRDD.getJavaRDD; - public abstract class HoodieSparkTable - extends HoodieTable>, JavaRDD, JavaRDD> { + extends HoodieTable>, HoodieData, HoodieData> { private volatile boolean isMetadataTableExists = false; @@ -81,7 +77,7 @@ public static HoodieSparkTable create(HoodieW HoodieSparkEngineContext context, HoodieTableMetaClient metaClient, boolean refreshTimeline) { - HoodieSparkTable hoodieSparkTable; + HoodieSparkTable hoodieSparkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: hoodieSparkTable = new HoodieSparkCopyOnWriteTable<>(config, context, metaClient); @@ -98,11 +94,6 @@ public static HoodieSparkTable create(HoodieW return hoodieSparkTable; } - public static HoodieWriteMetadata> convertMetadata( - HoodieWriteMetadata> metadata) { - return metadata.clone(getJavaRDD(metadata.getWriteStatuses())); - } - @Override protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndexFactory.createIndex(config); 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 a970e8f0f97d3..504da8a722810 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 @@ -28,10 +28,10 @@ import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator; import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.client.utils.SparkValidatorUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -68,7 +68,6 @@ 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.nio.charset.StandardCharsets; @@ -80,10 +79,11 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; import static org.apache.hudi.table.action.bootstrap.MetadataBootstrapHandlerFactory.getMetadataHandler; public class SparkBootstrapCommitActionExecutor> - extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { + extends BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieBootstrapWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); protected String bootstrapSchema = null; @@ -91,7 +91,7 @@ public class SparkBootstrapCommitActionExecutor public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, Option> extraMetadata) { super(context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) @@ -109,7 +109,7 @@ private void validate() { } @Override - public HoodieBootstrapWriteMetadata execute() { + public HoodieBootstrapWriteMetadata> execute() { validate(); try { HoodieTableMetaClient metaClient = table.getMetaClient(); @@ -121,9 +121,9 @@ public HoodieBootstrapWriteMetadata execute() { Map>>> partitionSelections = listAndProcessSourcePartitions(); // First run metadata bootstrap which will auto commit - Option metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); + Option>> metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); // if there are full bootstrap to be performed, perform that too - Option fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); + Option>> fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); // Delete the marker directory for the instant WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); @@ -142,7 +142,7 @@ protected String getSchemaToStoreInCommit() { * Perform Metadata Bootstrap. * @param partitionFilesList List of partitions and files within that partitions */ - protected Option metadataBootstrap(List>> partitionFilesList) { + protected Option>> metadataBootstrap(List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } @@ -155,43 +155,42 @@ protected Option metadataBootstrap(List bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); + HoodieData bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result); return Option.of(result); } - private void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata> result) { + private void updateIndexAndCommitIfNeeded(HoodieData writeStatuses, HoodieWriteMetadata> result) { // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future // RDD actions that are performed after updating the index. - writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = HoodieJavaRDD.getJavaRDD( - table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); commitOnAutoCommit(result); } @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecords) { + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { // NO_OP return null; } @Override - protected void setCommitMetadata(HoodieWriteMetadata> result) { + protected void setCommitMetadata(HoodieWriteMetadata> result) { result.setCommitMetadata(Option.of(new HoodieCommitMetadata())); } @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index // is all done in a single job DAG. Map>> bootstrapSourceAndStats = - result.getWriteStatuses().collect().stream() + result.getWriteStatuses().collectAsList().stream() .map(w -> { BootstrapWriteStatus ws = (BootstrapWriteStatus) w; return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); @@ -214,7 +213,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta LOG.info("Committing metadata bootstrap !!"); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { String actionType = table.getMetaClient().getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); // Create a Hoodie table which encapsulated the commits and files visible @@ -253,7 +252,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta * Perform Full Bootstrap. * @param partitionFilesList List of partitions and files within that partitions */ - protected Option fullBootstrap(List>> partitionFilesList) { + protected Option>> fullBootstrap(List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } @@ -271,10 +270,10 @@ protected Option fullBootstrap(List getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(HoodieData inputRecordsRDD) { return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, inputRecordsRDD, Option.empty(), extraMetadata); @@ -310,10 +309,9 @@ private Map>>> listAndPr .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } - private JavaRDD runMetadataBootstrap(List>> partitions) { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + private HoodieData runMetadataBootstrap(List>> partitions) { if (null == partitions || partitions.isEmpty()) { - return jsc.emptyRDD(); + return context.emptyHoodieData(); } TypedProperties properties = new TypedProperties(); @@ -336,7 +334,8 @@ private JavaRDD runMetadataBootstrap(List getMetadataHandler(config, table, partitionFsPair.getRight().getRight()).runMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), keyGenerator)); } @@ -352,7 +351,7 @@ protected Iterator> handleUpdate(String partitionPath, String } @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java index 59f86662b7c0c..d712ca430b6f7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java @@ -18,9 +18,8 @@ package org.apache.hudi.table.action.bootstrap; -import java.util.Map; - import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -29,7 +28,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; -import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; public class SparkBootstrapDeltaCommitActionExecutor> extends SparkBootstrapCommitActionExecutor { @@ -41,7 +41,7 @@ public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context, } @Override - protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(HoodieData inputRecordsRDD) { return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, inputRecordsRDD, extraMetadata); 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 index 594a910428aad..7d2a4c0baabe3 100644 --- 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 @@ -28,14 +28,11 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.spark.api.java.JavaRDD; - public class SparkExecuteClusteringCommitActionExecutor> extends BaseSparkCommitActionExecutor { @@ -52,10 +49,8 @@ public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, } @Override - public HoodieWriteMetadata> execute() { - HoodieWriteMetadata> writeMetadata = executeClustering(clusteringPlan); - JavaRDD transformedWriteStatuses = HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses()); - return writeMetadata.clone(transformedWriteStatuses); + public HoodieWriteMetadata> execute() { + return executeClustering(clusteringPlan); } @Override 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 ba3b0be1641ee..ade550897765a 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 @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.client.utils.SparkValidatorUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -37,6 +37,7 @@ 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.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; @@ -54,13 +55,13 @@ 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; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.storage.StorageLevel; -import scala.Tuple2; import java.io.IOException; import java.io.Serializable; @@ -76,10 +77,13 @@ import java.util.Set; import java.util.stream.Collectors; +import scala.Tuple2; + import static org.apache.hudi.common.util.ClusteringUtils.getAllFileGroupsInPendingClusteringPlans; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; public abstract class BaseSparkCommitActionExecutor extends - BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { + BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); protected final Option keyGeneratorOpt; @@ -97,7 +101,7 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, HoodieTable table, String instantTime, WriteOperationType operationType, - Option extraMetadata) { + Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); try { keyGeneratorOpt = config.populateMetaFields() @@ -108,14 +112,14 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, } } - private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { + private HoodieData> clusteringHandleUpdate(HoodieData> inputRecords) { context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering"); Set fileGroupsInPendingClustering = - table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); - UpdateStrategy updateStrategy = (UpdateStrategy) ReflectionUtils + table.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()); + UpdateStrategy>> updateStrategy = (UpdateStrategy>>) ReflectionUtils .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); - Pair>, Set> recordsAndPendingClusteringFileGroups = - (Pair>, Set>) updateStrategy.handleUpdate(inputRecordsRDD); + Pair>, Set> recordsAndPendingClusteringFileGroups = + updateStrategy.handleUpdate(inputRecords); Set fileGroupsWithUpdatesAndPendingClustering = recordsAndPendingClusteringFileGroups.getRight(); if (fileGroupsWithUpdatesAndPendingClustering.isEmpty()) { return recordsAndPendingClusteringFileGroups.getLeft(); @@ -138,20 +142,20 @@ private JavaRDD> clusteringHandleUpdate(JavaRDD> } @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { // Cache the tagged records, so we don't end up computing both // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling - if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) { - inputRecordsRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); + JavaRDD> inputRDD = HoodieJavaRDD.getJavaRDD(inputRecords); + if (inputRDD.getStorageLevel() == StorageLevel.NONE()) { + inputRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); } else { - LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel()); + LOG.info("RDD PreppedRecords was persisted at: " + inputRDD.getStorageLevel()); } WorkloadProfile workloadProfile = null; if (isWorkloadProfileNeeded()) { context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile"); - workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType, table.getIndex().canIndexLogFiles()); + workloadProfile = new WorkloadProfile(buildProfile(inputRecords), operationType, table.getIndex().canIndexLogFiles()); LOG.info("Input workload profile :" + workloadProfile); } @@ -162,30 +166,23 @@ public HoodieWriteMetadata> execute(JavaRDD } // handle records update with clustering - JavaRDD> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD); + HoodieData> inputRecordsWithClusteringUpdate = clusteringHandleUpdate(inputRecords); context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data"); - JavaRDD> partitionedRecords = partition(inputRecordsRDDWithClusteringUpdate, partitioner); - JavaRDD writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> { - if (WriteOperationType.isChangingRecords(operationType)) { - return handleUpsertPartition(instantTime, partition, recordItr, partitioner); - } else { - return handleInsertPartition(instantTime, partition, recordItr, partitioner); - } - }, true).flatMap(List::iterator); - - updateIndexAndCommitIfNeeded(writeStatusRDD, result); + HoodieData writeStatuses = mapPartitionsAsRDD(inputRecordsWithClusteringUpdate, partitioner); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + updateIndexAndCommitIfNeeded(writeStatuses, result); return result; } - private Pair, WorkloadStat> buildProfile(JavaRDD> inputRecordsRDD) { + private Pair, WorkloadStat> buildProfile(HoodieData> inputRecords) { HashMap partitionPathStatMap = new HashMap<>(); WorkloadStat globalStat = new WorkloadStat(); // group the records by partitionPath + currentLocation combination, count the number of // records in each partition - Map>, Long> partitionLocationCounts = inputRecordsRDD - .mapToPair(record -> new Tuple2<>( + Map>, Long> partitionLocationCounts = inputRecords + .mapToPair(record -> Pair.of( new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) .countByKey(); @@ -223,9 +220,9 @@ protected Partitioner getPartitioner(WorkloadProfile profile) { } } - private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { - JavaPairRDD>, HoodieRecord> mappedRDD = dedupedRecords.mapToPair( - record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)); + private HoodieData mapPartitionsAsRDD(HoodieData> dedupedRecords, Partitioner partitioner) { + JavaPairRDD>, HoodieRecord> mappedRDD = HoodieJavaPairRDD.getJavaPairRDD( + dedupedRecords.mapToPair(record -> Pair.of(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record))); JavaPairRDD>, HoodieRecord> partitionedRDD; if (table.requireSortedRecords()) { @@ -242,24 +239,28 @@ record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.ge // Partition only partitionedRDD = mappedRDD.partitionBy(partitioner); } - - return partitionedRDD.map(Tuple2::_2); + return HoodieJavaRDD.of(partitionedRDD.map(Tuple2::_2).mapPartitionsWithIndex((partition, recordItr) -> { + if (WriteOperationType.isChangingRecords(operationType)) { + return handleUpsertPartition(instantTime, partition, recordItr, partitioner); + } else { + return handleInsertPartition(instantTime, partition, recordItr, partitioner); + } + }, true).flatMap(List::iterator)); } - protected JavaRDD updateIndex(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + protected HoodieData updateIndex(HoodieData writeStatuses, HoodieWriteMetadata> result) { // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future // RDD actions that are performed after updating the index. - writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = HoodieJavaRDD.getJavaRDD( - table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; } - protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + protected void updateIndexAndCommitIfNeeded(HoodieData writeStatusRDD, HoodieWriteMetadata> result) { updateIndex(writeStatusRDD, result); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); commitOnAutoCommit(result); @@ -271,19 +272,19 @@ protected String getCommitActionType() { } @Override - protected void setCommitMetadata(HoodieWriteMetadata> result) { - result.setCommitMetadata(Option.of(CommitUtils.buildMetadata(result.getWriteStatuses().map(WriteStatus::getStat).collect(), + protected void setCommitMetadata(HoodieWriteMetadata> result) { + result.setCommitMetadata(Option.of(CommitUtils.buildMetadata(result.getWriteStatuses().map(WriteStatus::getStat).collectAsList(), result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()))); } @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + 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()); + commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collectAsList()); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List writeStats) { String actionType = getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType + ", operation Type " + operationType); result.setCommitted(true); @@ -304,7 +305,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta } } - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeStatuses) { + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeStatuses) { return Collections.emptyMap(); } @@ -341,20 +342,20 @@ public Iterator> handleUpdate(String partitionPath, String fil // 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(); + return Collections.emptyIterator(); } // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); 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( "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId); } else { - SparkMergeHelper.newInstance().runMerge(table, upsertHandle); + HoodieMergeHelper.newInstance().runMerge(table, upsertHandle); } // TODO(vc): This needs to be revisited @@ -383,9 +384,9 @@ public Iterator> handleInsert(String idPfx, Iterator) Collections.EMPTY_LIST).iterator(); + return Collections.emptyIterator(); } - return new SparkLazyInsertIterable(recordItr, true, config, instantTime, table, idPfx, + return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, taskContextSupplier, new CreateHandleFactory<>()); } @@ -393,7 +394,7 @@ 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); + return new UpsertPartitioner<>(profile, context, table, config); } public Partitioner getInsertPartitioner(WorkloadProfile profile) { @@ -407,7 +408,7 @@ public Partitioner getLayoutPartitioner(WorkloadProfile profile, String layoutPa } @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); } } 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 f4f1d3ad06ccf..f4b01c887b068 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 @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -30,24 +31,22 @@ 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; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> inputRecordsRDD; + private final Option bulkInsertPartitioner; public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner, + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; @@ -55,7 +54,7 @@ public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, Hoo } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); 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 d0c5ddef5e71d..38e38101b0d02 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 @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -26,6 +27,7 @@ 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.data.HoodieJavaRDD; import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction; import org.apache.hudi.io.CreateHandleFactory; @@ -46,28 +48,28 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, - JavaRDD, JavaRDD, R> { +public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, + HoodieData, HoodieData, R> { private SparkBulkInsertHelper() { } private static class BulkInsertHelperHolder { - private static final SparkBulkInsertHelper SPARK_BULK_INSERT_HELPER = new SparkBulkInsertHelper(); + private static final SparkBulkInsertHelper HOODIE_BULK_INSERT_HELPER = new SparkBulkInsertHelper<>(); } public static SparkBulkInsertHelper newInstance() { - return BulkInsertHelperHolder.SPARK_BULK_INSERT_HELPER; + return BulkInsertHelperHolder.HOODIE_BULK_INSERT_HELPER; } @Override - 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) { + public HoodieWriteMetadata> bulkInsert(final HoodieData> inputRecords, + final String instantTime, + final HoodieTable>, HoodieData, HoodieData> table, + final HoodieWriteConfig config, + final BaseCommitActionExecutor>, HoodieData, HoodieData, R> executor, + final boolean performDedupe, + final Option userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); //transition bulk_insert state to inflight @@ -75,7 +77,7 @@ public HoodieWriteMetadata> bulkInsert(final JavaRDD writeStatuses = + HoodieData writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), new CreateHandleFactory(false)); //update index ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); @@ -83,39 +85,40 @@ public HoodieWriteMetadata> bulkInsert(final JavaRDD bulkInsert(JavaRDD> inputRecords, - String instantTime, - HoodieTable>, JavaRDD, JavaRDD> table, - HoodieWriteConfig config, - boolean performDedupe, - Option>>> userDefinedBulkInsertPartitioner, - boolean useWriterSchema, - int parallelism, - WriteHandleFactory writeHandleFactory) { + public HoodieData bulkInsert(HoodieData> inputRecords, + String instantTime, + HoodieTable>, HoodieData, HoodieData> table, + HoodieWriteConfig config, + boolean performDedupe, + Option userDefinedBulkInsertPartitioner, + boolean useWriterSchema, + int parallelism, + WriteHandleFactory writeHandleFactory) { // De-dupe/merge if needed - JavaRDD> dedupedRecords = inputRecords; + HoodieData> dedupedRecords = inputRecords; if (performDedupe) { - dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, + dedupedRecords = (HoodieData>) HoodieWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, parallelism, table); } - final JavaRDD> repartitionedRecords; + final HoodieData> repartitionedRecords; BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); - repartitionedRecords = (JavaRDD>) partitioner.repartitionRecords(dedupedRecords, parallelism); + // only JavaRDD is supported for Spark partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463 + repartitionedRecords = HoodieJavaRDD.of((JavaRDD>) partitioner.repartitionRecords(HoodieJavaRDD.getJavaRDD(dedupedRecords), parallelism)); // generate new file ID prefixes for each output partition final List fileIDPrefixes = IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); - JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, + JavaRDD writeStatusRDD = HoodieJavaRDD.getJavaRDD(repartitionedRecords) + .mapPartitionsWithIndex(new BulkInsertMapFunction<>(instantTime, partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, useWriterSchema, writeHandleFactory), true) .flatMap(List::iterator); - return writeStatusRDD; + return HoodieJavaRDD.of(writeStatusRDD); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java index 28d8cb0b26422..8862981c2a2b7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -30,25 +31,23 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkBulkInsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedInputRecordRdd; - private final Option>>> userDefinedBulkInsertPartitioner; + private final HoodieData> preppedInputRecordRdd; + private final Option userDefinedBulkInsertPartitioner; public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option>>> userDefinedBulkInsertPartitioner) { + String instantTime, HoodieData> preppedInputRecordRdd, + Option userDefinedBulkInsertPartitioner) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, userDefinedBulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java index 997c7bf2376e3..a6fc996b71c31 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.spark.api.java.JavaRDD; public class SparkDeleteCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD keys; + private final HoodieData keys; public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { + String instantTime, HoodieData keys) { super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + public HoodieWriteMetadata> execute() { + return HoodieDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } 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 index 90bcdc9b9c141..b31eb7b96d948 100644 --- 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 @@ -19,20 +19,18 @@ 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.data.HoodieData; 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.data.HoodieJavaPairRDD; 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; @@ -51,16 +49,15 @@ public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, } @Override - public HoodieWriteMetadata> execute() { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + public HoodieWriteMetadata> execute() { 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(); + context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions."); + Map> partitionToReplaceFileIds = HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitions).distinct() + .mapToPair(partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); - - result.setWriteStatuses(jsc.emptyRDD()); + result.setWriteStatuses(context.emptyHoodieData()); 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/SparkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java index ba91fe1607916..479b51322ff32 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.spark.api.java.JavaRDD; public class SparkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } 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 7a3549c9e9c41..518063ed34186 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,18 +19,21 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.FileSlice; 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.timeline.HoodieTimeline; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaRDD; -import scala.Tuple2; import java.util.List; import java.util.Map; @@ -39,25 +42,25 @@ public class SparkInsertOverwriteCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { this(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE); } public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, + String instantTime, HoodieData> inputRecordsRDD, WriteOperationType writeOperationType) { super(context, config, table, instantTime, writeOperationType); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } @@ -74,13 +77,13 @@ protected String getCommitActionType() { } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - return writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> - new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + return HoodieJavaPairRDD.getJavaPairRDD(writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> + Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); } 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()); + return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList()); } } 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 index f7c98d5373360..93d0a8124c4ee 100644 --- 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 @@ -19,20 +19,19 @@ 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.data.HoodieData; 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.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.table.HoodieTable; 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.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -41,21 +40,18 @@ public class SparkInsertOverwriteTableCommitActionExecutor> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE); } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - Map> partitionToExistingFileIds = new HashMap<>(); + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { 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(); + if (partitionPaths == null || partitionPaths.isEmpty()) { + return Collections.emptyMap(); } - return partitionToExistingFileIds; + context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); + return HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitionPaths, partitionPaths.size()).mapToPair( + partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java index 400147bb8fe70..ff1a7e2b9beeb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.spark.api.java.JavaRDD; public class SparkInsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java index c914384cb21ed..ccee9cf5a7164 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.spark.api.java.JavaRDD; public class SparkUpsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java index e36073fd17d6d..73d408593bd37 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.spark.api.java.JavaRDD; public class SparkUpsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } 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 6ca4408a7bba2..61cb1ffd27bd1 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 @@ -19,7 +19,6 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -27,10 +26,9 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; /** * Compacts a hoodie table with merge on read storage. Computes all possible compactions, @@ -39,7 +37,7 @@ */ @SuppressWarnings("checkstyle:LineLength") public class HoodieSparkMergeOnReadTableCompactor - extends HoodieCompactor>, JavaRDD, JavaRDD> { + extends HoodieCompactor>, HoodieData, HoodieData> { @Override public void preCompact( @@ -53,6 +51,6 @@ public void preCompact( @Override public void maybePersist(HoodieData writeStatus, HoodieWriteConfig config) { - HoodieJavaRDD.getJavaRDD(writeStatus).persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatus.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java index 222506e7bbb36..61e6f25af9429 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.deltacommit; -import java.util.Map; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; @@ -42,13 +41,14 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; public abstract class BaseSparkDeltaCommitActionExecutor> extends BaseSparkCommitActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseSparkDeltaCommitActionExecutor.class); // UpsertPartitioner for MergeOnRead table type - private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; + private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; public BaseSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType) { @@ -66,7 +66,7 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { if (profile == null) { throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); } - mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config); + mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner<>(profile, (HoodieSparkEngineContext) context, table, config); return mergeOnReadUpsertPartitioner; } @@ -79,7 +79,7 @@ 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(); return Collections.singletonList(appendHandle.close()).iterator(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java index 6f23e41773bbd..190a714e44612 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,25 +32,23 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - import java.util.Map; public class SparkBulkInsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> inputRecordsRDD; + private final Option bulkInsertPartitioner; public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner, + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; @@ -57,7 +56,7 @@ public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java index be5b903c7642d..c01bce2b9cf35 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,25 +32,23 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - public class SparkBulkInsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedInputRecordRdd; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> preppedInputRecordRdd; + private final Option bulkInsertPartitioner; public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> preppedInputRecordRdd, + Option bulkInsertPartitioner) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecordRdd = preppedInputRecordRdd; this.bulkInsertPartitioner = bulkInsertPartitioner; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, bulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java index 7cff563571459..9a5b08df288d6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.hudi.table.action.commit.SparkDeleteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieDeleteHelper; public class SparkDeleteDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD keys; + private final HoodieData keys; public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { + String instantTime, HoodieData keys) { super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + public HoodieWriteMetadata> execute() { + return HoodieDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java index 7e38823fc8838..4889460c467fa 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java @@ -20,31 +20,30 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.hudi.table.action.commit.SparkWriteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; public class SparkInsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java index e401d9555e434..dbf0cbc676118 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -27,22 +28,20 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkInsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java index c63be6289004d..67ecb9a8cbc06 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java @@ -18,32 +18,32 @@ package org.apache.hudi.table.action.deltacommit; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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 org.apache.hudi.table.action.commit.SparkWriteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; public class SparkUpsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java index f593fea779029..9540030eba157 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -27,22 +28,20 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkUpsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 6e67bd69bdfd7..ce0cc37c78e51 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -32,6 +32,7 @@ import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator; import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; @@ -76,6 +77,7 @@ import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; @@ -83,17 +85,19 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; +import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; 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.commit.SparkWriteHelper; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -451,13 +455,13 @@ private void testDeduplication( HoodieRecord recordThree = new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); - JavaRDD> records = - jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); + HoodieData> records = HoodieJavaRDD.of( + jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1)); // Global dedup should be done based on recordKey only HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(true); - List> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + List> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList(); assertEquals(1, dedupedRecs.size()); assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath()); assertNodupesWithinPartition(dedupedRecs); @@ -465,7 +469,7 @@ private void testDeduplication( // non-Global dedup should be done based on both recordKey and partitionPath index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(false); - dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList(); assertEquals(2, dedupedRecs.size()); assertNodupesWithinPartition(dedupedRecs); @@ -779,6 +783,20 @@ private void testHoodieConcatHandleOnDupInserts(HoodieWriteConfig config, boolea firstInsertRecords + secondInsertRecords, 2, false, config.populateMetaFields()); } + @Test + public void testBulkInsertWithCustomPartitioner() { + HoodieWriteConfig config = getConfigBuilder().withRollbackUsingMarkers(true).build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { + final String commitTime1 = "001"; + client.startCommitWithTime(commitTime1); + List inserts1 = dataGen.generateInserts(commitTime1, 100); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 10); + BulkInsertPartitioner> partitioner = new RDDCustomColumnsSortPartitioner(new String[]{"rider"}, HoodieTestDataGenerator.AVRO_SCHEMA, false); + List statuses = client.bulkInsert(insertRecordsRDD1, commitTime1, Option.of(partitioner)).collect(); + assertNoWriteErrors(statuses); + } + } + /** * Tests deletion of records. */ @@ -2594,7 +2612,7 @@ private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleani .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); } - public static class FailingPreCommitValidator> extends SparkPreCommitValidator { + public static class FailingPreCommitValidator> extends SparkPreCommitValidator { public FailingPreCommitValidator(HoodieSparkTable table, HoodieEngineContext context, HoodieWriteConfig config) { super(table, context, config); 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 dcc41addc8f31..b9f025223b7df 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 @@ -39,6 +39,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -556,7 +557,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { // initialize partitioner hoodieTable.getHoodieView().sync(); BaseSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable, - newDeleteTime, deleteRDD); + newDeleteTime, HoodieJavaRDD.of(deleteRDD)); actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); final List> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java index 53cd6e5d1e749..0b29cf25f9e3f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; @@ -38,6 +39,7 @@ import org.apache.hudi.config.HoodieLayoutConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.index.HoodieIndex; @@ -325,7 +327,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { // Insert new records BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - firstCommitTime, jsc.parallelize(records)); + firstCommitTime, context.parallelize(records)); List writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -368,7 +370,7 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs2 = records; BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs2)); + instantTime, context.parallelize(recs2)); List returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -389,7 +391,7 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs3 = records; BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs3)); + instantTime, context.parallelize(recs3)); returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -422,7 +424,7 @@ public void testFileSizeUpsertRecords() throws Exception { // Insert new records BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(records)); + instantTime, context.parallelize(records)); jsc.parallelize(Arrays.asList(1)) .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) .map(Transformations::flatten).collect(); @@ -452,7 +454,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { // Perform inserts of 100 records to test CreateHandle and BufferedExecutor final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100); BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(inserts)); + instantTime, context.parallelize(inserts)); final List> ws = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator()); }).map(Transformations::flatten).collect(); @@ -466,7 +468,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, HoodieTableMetaClient.reload(metaClient)); BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(updates)); + instantTime, context.parallelize(updates)); final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator()); }).map(Transformations::flatten).collect(); @@ -486,8 +488,8 @@ public void testBulkInsertRecords(String bulkInsertMode) throws Exception { // Insert new records final JavaRDD inputRecords = generateTestRecordsForBulkInsert(jsc); SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor( - context, config, table, instantTime, inputRecords, Option.empty()); - List returnedStatuses = ((JavaRDD)bulkInsertExecutor.execute().getWriteStatuses()).collect(); + context, config, table, instantTime, HoodieJavaRDD.of(inputRecords), Option.empty()); + List returnedStatuses = ((HoodieData) bulkInsertExecutor.execute().getWriteStatuses()).collectAsList(); verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords)); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java deleted file mode 100644 index 2d852f8107ef0..0000000000000 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ /dev/null @@ -1,194 +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.table.action.commit; - -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; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.HoodieWriteMetadata; - -import org.apache.spark.Partition; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; - -import java.util.Collections; -import java.util.List; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@ExtendWith(MockitoExtension.class) -public class TestDeleteHelper { - - private enum CombineTestMode { - None, GlobalIndex, NoneGlobalIndex; - } - - private static final String BASE_PATH = "/tmp/"; - private static final boolean WITH_COMBINE = true; - private static final boolean WITHOUT_COMBINE = false; - private static final int DELETE_PARALLELISM = 200; - - @Mock - private HoodieIndex index; - @Mock - private HoodieTable, JavaRDD, JavaRDD> table; - @Mock - private BaseSparkCommitActionExecutor executor; - @Mock - private HoodieWriteMetadata metadata; - @Mock - private JavaPairRDD keyPairs; - @Mock - private JavaSparkContext jsc; - @Mock - private HoodieSparkEngineContext context; - - private JavaRDD rddToDelete; - private HoodieWriteConfig config; - - @BeforeEach - public void setUp() { - when(table.getIndex()).thenReturn(index); - when(context.getJavaSparkContext()).thenReturn(jsc); - } - - @Test - public void deleteWithEmptyRDDShouldNotExecute() { - rddToDelete = mockEmptyHoodieKeyRdd(); - config = newWriteConfig(WITHOUT_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, never()).repartition(DELETE_PARALLELISM); - verifyNoDeleteExecution(); - } - - @Test - public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None); - config = newWriteConfig(WITHOUT_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM); - verifyDeleteExecution(); - } - - @Test - public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex); - config = newWriteConfig(WITH_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM); - verifyDeleteExecution(); - } - - @Test - public void deleteWithCombineShouldRepartitionForNonEmptyRddAndGlobalIndex() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.GlobalIndex); - config = newWriteConfig(WITH_COMBINE); - when(index.isGlobal()).thenReturn(true); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM)); - verifyDeleteExecution(); - } - - private void verifyDeleteExecution() { - verify(executor, times(1)).execute(any()); - verify(metadata, times(1)).setIndexLookupDuration(any()); - } - - private void verifyNoDeleteExecution() { - verify(executor, never()).execute(any()); - } - - private HoodieWriteConfig newWriteConfig(boolean combine) { - return HoodieWriteConfig.newBuilder() - .combineDeleteInput(combine) - .withPath(BASE_PATH) - .withDeleteParallelism(DELETE_PARALLELISM) - .build(); - } - - private JavaRDD newHoodieKeysRddMock(int howMany, CombineTestMode combineMode) { - JavaRDD keysToDelete = mock(JavaRDD.class); - - JavaRDD recordsRdd = mock(JavaRDD.class); - when(recordsRdd.filter(any())).thenReturn(recordsRdd); - when(recordsRdd.isEmpty()).thenReturn(howMany <= 0); - when(index.tagLocation(any(), any(), any())).thenReturn(HoodieJavaRDD.of(recordsRdd)); - - if (combineMode == CombineTestMode.GlobalIndex) { - when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs); - when(keyPairs.values()).thenReturn(keysToDelete); - when(keysToDelete.keyBy(any())).thenReturn(keyPairs); - } else if (combineMode == CombineTestMode.NoneGlobalIndex) { - when(keysToDelete.distinct(anyInt())).thenReturn(keysToDelete); - } else if (combineMode == CombineTestMode.None) { - List parts = mock(List.class); - when(parts.isEmpty()).thenReturn(howMany <= 0); - when(keysToDelete.repartition(anyInt())).thenReturn(keysToDelete); - when(keysToDelete.partitions()).thenReturn(parts); - } - - when(keysToDelete.map(any())).thenReturn(recordsRdd); - when(executor.execute(any())).thenReturn(metadata); - return keysToDelete; - } - - private JavaRDD mockEmptyHoodieKeyRdd() { - JavaRDD emptyRdd = mock(JavaRDD.class); - doReturn(true).when(emptyRdd).isEmpty(); - doReturn(Collections.emptyList()).when(emptyRdd).partitions(); - doReturn(emptyRdd).when(emptyRdd).map(any()); - - doReturn(HoodieJavaRDD.of(emptyRdd)).when(index).tagLocation(any(), any(), any()); - doReturn(emptyRdd).when(emptyRdd).filter(any()); - - doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString()); - doReturn(emptyRdd).when(jsc).emptyRDD(); - return emptyRdd; - } - -} 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 9afe5f3533cac..59174a9371a58 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 @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; @@ -195,12 +196,12 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { String compactionInstantTime = "102"; table.scheduleCompaction(context, compactionInstantTime, Option.empty()); table.getMetaClient().reloadActiveTimeline(); - JavaRDD result = (JavaRDD) table.compact( + HoodieData result = (HoodieData) table.compact( context, compactionInstantTime).getWriteStatuses(); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { - List writeStatuses = result.collect(); + List writeStatuses = result.collectAsList(); assertTrue(writeStatuses.stream() .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)).count() > 0); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java index 093fd439db09c..4e8d2b7eceaee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java @@ -97,6 +97,12 @@ public abstract HoodieData mapPartitions( */ public abstract HoodieData distinct(); + public abstract HoodieData distinct(int parallelism); + + public abstract HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism); + + public abstract HoodieData filter(SerializableFunction filterFunc); + /** * Unions this {@link HoodieData} with other {@link HoodieData}. * @param other {@link HoodieData} of interest. @@ -108,4 +114,6 @@ public abstract HoodieData mapPartitions( * @return collected results in {@link List}. */ public abstract List collectAsList(); + + public abstract HoodieData repartition(int parallelism); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java index 94416192abfb9..c23e712cf41ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java @@ -132,6 +132,26 @@ public HoodieData distinct() { return HoodieList.of(new ArrayList<>(new HashSet<>(listData))); } + @Override + public HoodieData distinct(int parallelism) { + return distinct(); + } + + @Override + public HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism) { + return mapToPair(i -> Pair.of(keyGetter.apply(i), i)) + .reduceByKey((value1, value2) -> value1, parallelism) + .values(); + } + + @Override + public HoodieData filter(SerializableFunction filterFunc) { + return HoodieList.of(listData + .stream() + .filter(i -> throwingMapWrapper(filterFunc).apply(i)) + .collect(Collectors.toList())); + } + @Override public HoodieData union(HoodieData other) { List unionResult = new ArrayList<>(); @@ -144,4 +164,10 @@ public HoodieData union(HoodieData other) { public List collectAsList() { return listData; } + + @Override + public HoodieData repartition(int parallelism) { + // no op + return this; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java index c941231e617f1..1e125c90a190b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.data; import org.apache.hudi.common.function.FunctionWrapper; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -27,6 +28,7 @@ import org.apache.hudi.common.util.collection.Pair; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -110,6 +112,15 @@ public Map countByKey() { Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size())); } + @Override + public HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism) { + return HoodieMapPair.of(mapPairData.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> { + Option reducedValue = Option.fromJavaOptional(e.getValue().stream().reduce(func::apply)); + return reducedValue.isPresent() ? Collections.singletonList(reducedValue.get()) : Collections.emptyList(); + }))); + } + @Override public HoodieData map(SerializableFunction, O> func) { Function, O> throwableFunc = throwingMapWrapper(func); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java index b9bdcb3d9ff4a..9ff52793d6f0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.data; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -72,6 +73,8 @@ public abstract class HoodiePairData implements Serializable { */ public abstract Map countByKey(); + public abstract HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism); + /** * @param func serializable map function. * @param output object type. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java new file mode 100644 index 0000000000000..6130d4af1094e --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.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.data; + +import org.apache.hudi.common.util.collection.Pair; + +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; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class TestHoodieList { + + private static Stream distinctWithKey() { + return Stream.of( + Arguments.of( + Arrays.asList(Pair.of("k1", 1), Pair.of("k2", 2)), + Arrays.asList(Pair.of("k1", 1), Pair.of("k1", 10), Pair.of("k1", 100), Pair.of("k2", 2))) + ); + } + + @ParameterizedTest + @MethodSource + void distinctWithKey(List> expected, List> originalList) { + List> distinctList = HoodieList.of(originalList).distinctWithKey(Pair::getLeft, 1).collectAsList(); + assertEquals(expected, distinctList); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java index 86b1a213ba639..20e9a8f5d9b13 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java @@ -25,6 +25,9 @@ import org.junit.jupiter.api.BeforeAll; 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.Arrays; @@ -33,7 +36,10 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.hudi.common.util.CollectionUtils.createImmutableList; +import static org.apache.hudi.common.util.CollectionUtils.createImmutableMap; import static org.junit.jupiter.api.Assertions.assertEquals; public class TestHoodieMapPair { @@ -117,6 +123,29 @@ public void testMapToPair() { }))); } + private static Stream testReduceByKey() { + return Stream.of( + Arguments.of( + createImmutableMap( + Pair.of(1, createImmutableList(1001)), + Pair.of(2, createImmutableList(2001)), + Pair.of(3, createImmutableList(3001)), + Pair.of(4, createImmutableList())), + createImmutableMap( + Pair.of(1, createImmutableList(1001, 1002, 1003)), + Pair.of(2, createImmutableList(2001, 2002)), + Pair.of(3, createImmutableList(3001)), + Pair.of(4, createImmutableList()))) + ); + } + + @ParameterizedTest + @MethodSource + public void testReduceByKey(Map> expected, Map> original) { + HoodiePairData reduced = HoodieMapPair.of(original).reduceByKey((a, b) -> a, 1); + assertEquals(expected, HoodieMapPair.getMapPair(reduced)); + } + @Test public void testLeftOuterJoinSingleValuePerKey() { HoodiePairData pairData1 = constructTestMapPairData(Arrays.asList( From 9ece77561a673fbfd666c5c57dac1a36e6cedce6 Mon Sep 17 00:00:00 2001 From: RexAn Date: Fri, 18 Mar 2022 03:53:45 +0800 Subject: [PATCH 21/33] [MINOR] HoodieFileScanRDD could print null path (#5056) Co-authored-by: Rex An --- .../src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala index 7e8f62bd2500a..aef6fc06222e7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala @@ -47,8 +47,8 @@ class HoodieFileScanRDD(@transient private val sparkSession: SparkSession, /** Advances to the next file. Returns true if a new non-empty iterator is available. */ private def nextIterator(): Boolean = { if (files.hasNext) { - logInfo(s"Reading File $currentFile") currentFile = files.next() + logInfo(s"Reading File $currentFile") currentIterator = readFunction(currentFile) try { From 6fe4d6e2f645db47b94f5c4d7743747308ef3d86 Mon Sep 17 00:00:00 2001 From: JerryYue-M <272614347@qq.com> Date: Fri, 18 Mar 2022 10:47:29 +0800 Subject: [PATCH 22/33] [HUDI-3598] Row Data to Hoodie Record Operator parallelism needs to always be consistent with input operator (#5049) for chaining purpose Co-authored-by: jerryyue --- .../src/main/java/org/apache/hudi/sink/utils/Pipelines.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index ae8b4f21300a2..0a5bb6cc3ef05 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -246,7 +246,8 @@ private static DataStream boundedBootstrap( * Transforms the row data to hoodie records. */ public static DataStream rowDataToHoodieRecord(Configuration conf, RowType rowType, DataStream dataStream) { - return dataStream.map(RowDataToHoodieFunctions.create(rowType, conf), TypeInformation.of(HoodieRecord.class)); + return dataStream.map(RowDataToHoodieFunctions.create(rowType, conf), TypeInformation.of(HoodieRecord.class)) + .setParallelism(dataStream.getParallelism()).name("row_data_to_hoodie_record"); } /** From 2551c26183cd11d47598128dfd08811fd3d83c69 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Fri, 18 Mar 2022 09:44:56 -0700 Subject: [PATCH 23/33] [HUDI-3656] Adding medium sized dataset for clustering and minor fixes to integ tests (#5063) --- .../deltastreamer-medium-clustering.yaml | 73 +++++++++++++++++++ .../test-suite/spark-medium-clustering.yaml | 59 +++++++++++++++ .../dag/nodes/ValidateAsyncOperations.java | 35 +++++---- 3 files changed, 152 insertions(+), 15 deletions(-) create mode 100644 docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml create mode 100644 docker/demo/config/test-suite/spark-medium-clustering.yaml diff --git a/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml new file mode 100644 index 0000000000000..0cd4108cb6334 --- /dev/null +++ b/docker/demo/config/test-suite/deltastreamer-medium-clustering.yaml @@ -0,0 +1,73 @@ +# 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. + +# to be used with test-aggressive-clean-archival.properties + +dag_name: deltastreamer-medium-clustering.yaml +dag_rounds: 20 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 1000 + num_partitions_insert: 50 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 1000 + num_partitions_insert: 2 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 2 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: third_insert + first_delete: + config: + num_partitions_delete: 50 + num_records_delete: 8000 + type: DeleteNode + deps: first_upsert + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete + last_validate: + config: + execute_itr_count: 20 + type: ValidateAsyncOperations + deps: second_validate diff --git a/docker/demo/config/test-suite/spark-medium-clustering.yaml b/docker/demo/config/test-suite/spark-medium-clustering.yaml new file mode 100644 index 0000000000000..09537a23d553e --- /dev/null +++ b/docker/demo/config/test-suite/spark-medium-clustering.yaml @@ -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. +dag_name: spark-medium-clustering.yaml +dag_rounds: 20 +dag_intermittent_delay_mins: 0 +dag_content: + first_insert: + config: + record_size: 200 + num_partitions_insert: 50 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: none + first_validate: + config: + validate_hive: false + type: ValidateDatasetNode + deps: first_insert + first_upsert: + config: + record_size: 200 + num_partitions_insert: 50 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 3000 + num_partitions_upsert: 50 + type: SparkUpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 50 + num_records_delete: 8000 + type: SparkDeleteNode + deps: first_upsert + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete + last_validate: + config: + execute_itr_count: 20 + type: ValidateAsyncOperations + deps: second_validate diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java index a7c3245f69921..7b5d4dd41881d 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java @@ -18,8 +18,15 @@ package org.apache.hudi.integ.testsuite.dag.nodes; +import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.dag.ExecutionContext; @@ -59,21 +66,19 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E int maxCommitsRetained = executionContext.getHoodieTestSuiteWriter().getWriteConfig().getCleanerCommitsRetained() + 1; FileSystem fs = FSUtils.getFs(basePath, executionContext.getHoodieTestSuiteWriter().getConfiguration()); - Map fileIdCount = new HashMap<>(); - - AtomicInteger maxVal = new AtomicInteger(); - List partitionPaths = FSUtils.getAllPartitionFoldersThreeLevelsDown(fs, basePath); - for (String partitionPath : partitionPaths) { - List fileStatuses = Arrays.stream(FSUtils.getAllDataFilesInPartition(fs, new Path(basePath + "/" + partitionPath))).collect(Collectors.toList()); - fileStatuses.forEach(entry -> { - String fileId = FSUtils.getFileId(entry.getPath().getName()); - fileIdCount.computeIfAbsent(fileId, k -> 0); - fileIdCount.put(fileId, fileIdCount.get(fileId) + 1); - maxVal.set(Math.max(maxVal.get(), fileIdCount.get(fileId))); - }); - } - if (maxVal.get() > maxCommitsRetained) { - throw new AssertionError("Total commits (" + maxVal + ") retained exceeds max value of " + maxCommitsRetained + ", total commits : "); + + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath) + .setConf(executionContext.getJsc().hadoopConfiguration()).build(); + Option latestCleanInstant = metaClient.getActiveTimeline().filter(instant -> instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)).lastInstant(); + if (latestCleanInstant.isPresent()) { + log.warn("Latest clean commit " + latestCleanInstant.get()); + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(metaClient, latestCleanInstant.get()); + String earliestCommitToRetain = cleanMetadata.getEarliestCommitToRetain(); + log.warn("Earliest commit to retain : " + earliestCommitToRetain); + long unCleanedInstants = metaClient.getActiveTimeline().filterCompletedInstants().filter(instant -> + HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestCommitToRetain)).getInstants().count(); + ValidationUtils.checkArgument(unCleanedInstants >= (maxCommitsRetained + 1), "Total uncleaned instants " + unCleanedInstants + + " mismatched with max commits retained " + (maxCommitsRetained + 1)); } if (config.validateArchival() || config.validateClean()) { From 316e38c71e2fec9d7a13417d409374f28875c4f1 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Fri, 18 Mar 2022 09:45:33 -0700 Subject: [PATCH 24/33] [HUDI-3659] Reducing the validation frequency with integ tests (#5067) --- .../test-suite/cow-spark-long-running.yaml | 8 +- ...er-long-running-multi-partitions-hive.yaml | 8 +- ...ong-running-multi-partitions-metadata.yaml | 1 + ...treamer-long-running-multi-partitions.yaml | 1 + ...reamer-medium-full-dataset-validation.yaml | 1 + .../detlastreamer-long-running-example.yaml | 8 +- .../testsuite/configuration/DeltaConfig.java | 5 + .../dag/nodes/BaseValidateDatasetNode.java | 143 +++++++++--------- 8 files changed, 87 insertions(+), 88 deletions(-) diff --git a/docker/demo/config/test-suite/cow-spark-long-running.yaml b/docker/demo/config/test-suite/cow-spark-long-running.yaml index c25b95c8da457..795a4a5f60709 100644 --- a/docker/demo/config/test-suite/cow-spark-long-running.yaml +++ b/docker/demo/config/test-suite/cow-spark-long-running.yaml @@ -25,11 +25,6 @@ dag_content: num_records_insert: 10000 type: SparkInsertNode deps: none - first_validate: - config: - validate_hive: false - type: ValidateDatasetNode - deps: first_insert first_upsert: config: record_size: 200 @@ -39,7 +34,7 @@ dag_content: num_records_upsert: 3000 num_partitions_upsert: 50 type: SparkUpsertNode - deps: first_validate + deps: first_insert first_delete: config: num_partitions_delete: 50 @@ -48,6 +43,7 @@ dag_content: deps: first_upsert second_validate: config: + validate_once_every_itr : 5 validate_hive: false delete_input_data: true type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml index 68d14a02dc54b..09dd6168c985e 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-hive.yaml @@ -47,11 +47,6 @@ dag_content: engine: "mr" type: HiveSyncNode deps: third_insert - first_validate: - config: - validate_hive: false - type: ValidateDatasetNode - deps: first_hive_sync first_upsert: config: record_size: 1000 @@ -61,7 +56,7 @@ dag_content: num_records_upsert: 100 num_partitions_upsert: 1 type: UpsertNode - deps: first_validate + deps: first_hive_sync first_delete: config: num_partitions_delete: 50 @@ -76,6 +71,7 @@ dag_content: deps: first_delete second_validate: config: + validate_once_every_itr : 5 validate_hive: true delete_input_data: true type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml index 0212fdf43c512..b2ab525b1af65 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions-metadata.yaml @@ -59,6 +59,7 @@ dag_content: deps: first_upsert second_validate: config: + validate_once_every_itr : 5 validate_hive: false delete_input_data: true type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml index d7b11194736e9..b8f2b686066c3 100644 --- a/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml +++ b/docker/demo/config/test-suite/deltastreamer-long-running-multi-partitions.yaml @@ -59,6 +59,7 @@ dag_content: deps: first_upsert second_validate: config: + validate_once_every_itr : 5 validate_hive: false delete_input_data: true type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml index 77898640ea144..a20870f262d8b 100644 --- a/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml +++ b/docker/demo/config/test-suite/deltastreamer-medium-full-dataset-validation.yaml @@ -62,6 +62,7 @@ dag_content: deps: first_upsert second_validate: config: + validate_once_every_itr : 5 validate_hive: false delete_input_data: false type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml index 4b2ee7ad13cc4..1c2f44b060036 100644 --- a/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml +++ b/docker/demo/config/test-suite/detlastreamer-long-running-example.yaml @@ -41,11 +41,6 @@ dag_content: num_records_insert: 300 deps: second_insert type: InsertNode - first_validate: - config: - validate_hive: false - type: ValidateDatasetNode - deps: third_insert first_upsert: config: record_size: 1000 @@ -55,7 +50,7 @@ dag_content: num_records_upsert: 100 num_partitions_upsert: 1 type: UpsertNode - deps: first_validate + deps: third_insert first_delete: config: num_partitions_delete: 1 @@ -64,6 +59,7 @@ dag_content: deps: first_upsert second_validate: config: + validate_once_every_itr : 5 validate_hive: false delete_input_data: true type: ValidateDatasetNode 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 2c39f5f93a52c..d7280402d2d5d 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 @@ -89,6 +89,7 @@ public static class Config { private static String START_PARTITION = "start_partition"; private static String DELETE_INPUT_DATA = "delete_input_data"; private static String VALIDATE_HIVE = "validate_hive"; + private static String VALIDATE_ONCE_EVERY_ITR = "validate_once_every_itr"; private static String EXECUTE_ITR_COUNT = "execute_itr_count"; private static String VALIDATE_ARCHIVAL = "validate_archival"; private static String VALIDATE_CLEAN = "validate_clean"; @@ -216,6 +217,10 @@ public boolean isValidateHive() { return Boolean.valueOf(configsMap.getOrDefault(VALIDATE_HIVE, false).toString()); } + public int validateOnceEveryIteration() { + return Integer.valueOf(configsMap.getOrDefault(VALIDATE_ONCE_EVERY_ITR, 1).toString()); + } + public boolean isValidateFullData() { return Boolean.valueOf(configsMap.getOrDefault(VALIDATE_FULL_DATA, false).toString()); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java index 09d44d986e183..de58bf6a1f205 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java @@ -74,81 +74,84 @@ public abstract Dataset getDatasetToValidate(SparkSession session, Executio @Override public void execute(ExecutionContext context, int curItrCount) 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 + "/*/*"; - log.warn("Validation using data from input path " + inputPath); - // listing batches to be validated - String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; - if (log.isDebugEnabled()) { - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); - FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); - log.info("fileStatuses length: " + fileStatuses.length); - for (FileStatus fileStatus : fileStatuses) { - log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); - } - } - - Dataset inputSnapshotDf = getInputDf(context, session, inputPath); - - // read from hudi and remove meta columns. - Dataset trimmedHudiDf = getDatasetToValidate(session, context, inputSnapshotDf.schema()); - if (config.isValidateFullData()) { - log.debug("Validating full dataset"); - Dataset exceptInputDf = inputSnapshotDf.except(trimmedHudiDf); - Dataset exceptHudiDf = trimmedHudiDf.except(inputSnapshotDf); - long exceptInputCount = exceptInputDf.count(); - long exceptHudiCount = exceptHudiDf.count(); - log.debug("Except input df count " + exceptInputDf + ", except hudi count " + exceptHudiCount); - if (exceptInputCount != 0 || exceptHudiCount != 0) { - log.error("Data set validation failed. Total count in hudi " + trimmedHudiDf.count() + ", input df count " + inputSnapshotDf.count() - + ". InputDf except hudi df = " + exceptInputCount + ", Hudi df except Input df " + exceptHudiCount); - throw new AssertionError("Hudi contents does not match contents input data. "); - } - } else { - Dataset intersectionDf = inputSnapshotDf.intersect(trimmedHudiDf); - long inputCount = inputSnapshotDf.count(); - long outputCount = trimmedHudiDf.count(); - log.debug("Input count: " + inputCount + "; output count: " + outputCount); - // the intersected df should be same as inputDf. if not, there is some mismatch. - if (outputCount == 0 || inputCount == 0 || inputSnapshotDf.except(intersectionDf).count() != 0) { - log.error("Data set validation failed. Total count in hudi " + outputCount + ", input df count " + inputCount); - throw new AssertionError("Hudi contents does not match contents input data. "); + int validateOnceEveryItr = config.validateOnceEveryIteration(); + int itrCountToExecute = config.getIterationCountToExecute(); + if ((itrCountToExecute != -1 && itrCountToExecute == curItrCount) || + (itrCountToExecute == -1 && ((curItrCount % validateOnceEveryItr) == 0))) { + 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 + "/*/*"; + log.warn("Validation using data from input path " + inputPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + if (log.isDebugEnabled()) { + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + log.info("fileStatuses length: " + fileStatuses.length); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } } - if (config.isValidateHive()) { - String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE().key()); - String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE().key()); - log.warn("Validating hive table with db : " + database + " and table : " + tableName); - session.sql("REFRESH TABLE " + database + "." + tableName); - Dataset cowDf = session.sql("SELECT _row_key, rider, driver, begin_lat, begin_lon, end_lat, end_lon, fare, _hoodie_is_deleted, " + - "test_suite_source_ordering_field FROM " + database + "." + tableName); - Dataset reorderedInputDf = inputSnapshotDf.select("_row_key", "rider", "driver", "begin_lat", "begin_lon", "end_lat", "end_lon", "fare", - "_hoodie_is_deleted", "test_suite_source_ordering_field"); - - Dataset intersectedHiveDf = reorderedInputDf.intersect(cowDf); - outputCount = trimmedHudiDf.count(); - log.warn("Input count: " + inputCount + "; output count: " + outputCount); + Dataset inputSnapshotDf = getInputDf(context, session, inputPath); + + // read from hudi and remove meta columns. + Dataset trimmedHudiDf = getDatasetToValidate(session, context, inputSnapshotDf.schema()); + if (config.isValidateFullData()) { + log.debug("Validating full dataset"); + Dataset exceptInputDf = inputSnapshotDf.except(trimmedHudiDf); + Dataset exceptHudiDf = trimmedHudiDf.except(inputSnapshotDf); + long exceptInputCount = exceptInputDf.count(); + long exceptHudiCount = exceptHudiDf.count(); + log.debug("Except input df count " + exceptInputDf + ", except hudi count " + exceptHudiCount); + if (exceptInputCount != 0 || exceptHudiCount != 0) { + log.error("Data set validation failed. Total count in hudi " + trimmedHudiDf.count() + ", input df count " + inputSnapshotDf.count() + + ". InputDf except hudi df = " + exceptInputCount + ", Hudi df except Input df " + exceptHudiCount); + throw new AssertionError("Hudi contents does not match contents input data. "); + } + } else { + Dataset intersectionDf = inputSnapshotDf.intersect(trimmedHudiDf); + long inputCount = inputSnapshotDf.count(); + long outputCount = trimmedHudiDf.count(); + log.debug("Input count: " + inputCount + "; output count: " + outputCount); // the intersected df should be same as inputDf. if not, there is some mismatch. - if (outputCount == 0 || reorderedInputDf.except(intersectedHiveDf).count() != 0) { - log.error("Data set validation failed for COW hive table. Total count in hudi " + outputCount + ", input df count " + inputCount); - throw new AssertionError("Hudi hive table contents does not match contents input data. "); + if (outputCount == 0 || inputCount == 0 || inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed. Total count in hudi " + outputCount + ", input df count " + inputCount); + throw new AssertionError("Hudi 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; - FileSystem fs = new Path(inputPathStr) - .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); - FileStatus[] 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); + if (config.isValidateHive()) { + String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE().key()); + String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE().key()); + log.warn("Validating hive table with db : " + database + " and table : " + tableName); + session.sql("REFRESH TABLE " + database + "." + tableName); + Dataset cowDf = session.sql("SELECT _row_key, rider, driver, begin_lat, begin_lon, end_lat, end_lon, fare, _hoodie_is_deleted, " + + "test_suite_source_ordering_field FROM " + database + "." + tableName); + Dataset reorderedInputDf = inputSnapshotDf.select("_row_key", "rider", "driver", "begin_lat", "begin_lon", "end_lat", "end_lon", "fare", + "_hoodie_is_deleted", "test_suite_source_ordering_field"); + + Dataset intersectedHiveDf = reorderedInputDf.intersect(cowDf); + outputCount = trimmedHudiDf.count(); + log.warn("Input count: " + inputCount + "; output count: " + outputCount); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (outputCount == 0 || reorderedInputDf.except(intersectedHiveDf).count() != 0) { + log.error("Data set validation failed for COW hive table. Total count in hudi " + outputCount + ", input df count " + inputCount); + 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; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] 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); + } } } } From 099c2c099ad842f18b76bc2529b6eeb968f328eb Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 18 Mar 2022 22:32:16 -0700 Subject: [PATCH 25/33] [HUDI-3457] Refactored Spark DataSource Relations to avoid code duplication (#4877) Refactoring Spark DataSource Relations to avoid code duplication. Following Relations were in scope: - BaseFileOnlyViewRelation - MergeOnReadSnapshotRelaation - MergeOnReadIncrementalRelation --- .../apache/hudi/HoodieConversionUtils.scala | 29 ++ .../org/apache/hudi/HoodieSparkUtils.scala | 5 - .../apache/hudi/BaseHoodieTableFileIndex.java | 2 +- .../common/table/TableSchemaResolver.java | 19 +- .../utils/HoodieRealtimeInputFormatUtils.java | 53 --- .../apache/hudi/BaseFileOnlyRelation.scala | 94 +++++ .../hudi/BaseFileOnlyViewRelation.scala | 141 ------- .../scala/org/apache/hudi/DefaultSource.scala | 2 +- .../org/apache/hudi/HoodieBaseRelation.scala | 204 ++++++++-- .../apache/hudi/HoodieBootstrapRelation.scala | 3 +- .../apache/hudi/HoodieDataSourceHelper.scala | 14 - .../org/apache/hudi/HoodieFileIndex.scala | 2 +- .../org/apache/hudi/HoodieFileScanRDD.scala | 6 +- .../apache/hudi/HoodieMergeOnReadRDD.scala | 2 +- .../hudi/MergeOnReadIncrementalRelation.scala | 278 ++++++------- .../hudi/MergeOnReadSnapshotRelation.scala | 167 +++----- .../hudi/SparkHoodieTableFileIndex.scala | 5 +- .../apache/spark/HoodieHadoopFSUtils.scala | 370 ++++++++++++++++++ .../datasources/HoodieInMemoryFileIndex.scala | 102 +++++ .../TestHoodieInMemoryFileIndex.scala | 60 +++ .../apache/hudi/TestHoodieSparkUtils.scala | 29 -- .../TestParquetColumnProjection.scala | 32 +- 22 files changed, 1049 insertions(+), 570 deletions(-) create mode 100644 hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieConversionUtils.scala create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala delete mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieHadoopFSUtils.scala create mode 100644 hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala create mode 100644 hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/TestHoodieInMemoryFileIndex.scala diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieConversionUtils.scala new file mode 100644 index 0000000000000..eaaf82182a77e --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieConversionUtils.scala @@ -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 + +object HoodieConversionUtils { + + def toJavaOption[T](opt: Option[T]): org.apache.hudi.common.util.Option[T] = + if (opt.isDefined) org.apache.hudi.common.util.Option.of(opt.get) else org.apache.hudi.common.util.Option.empty() + + def toScalaOption[T](opt: org.apache.hudi.common.util.Option[T]): Option[T] = + if (opt.isPresent) Some(opt.get) else None + +} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index c963806416061..cce6eacb03d73 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -118,11 +118,6 @@ object HoodieSparkUtils extends SparkAdapterSupport { }) } - def createInMemoryFileIndex(sparkSession: SparkSession, globbedPaths: Seq[Path]): InMemoryFileIndex = { - val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) - new InMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache) - } - /** * @deprecated please use other overload [[createRdd]] */ diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index 9eae74e928c25..8c88bfb001fc2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -198,7 +198,7 @@ protected HoodieTimeline getActiveTimeline() { // that is under the pending compaction process, new log-file will bear the compaction's instant (on the // timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering // such log-file we have to _always_ include pending compaction instants into consideration - // TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here + // TODO(HUDI-3302) re-evaluate whether we should filter any commits in here HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline(); if (shouldIncludePendingCommits) { return timeline; 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 a0771d1249c75..2c107694a1b28 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 @@ -509,19 +509,16 @@ public MessageType readSchemaFromLogFile(Path path) throws IOException { * @return */ public static MessageType readSchemaFromLogFile(FileSystem fs, Path path) throws IOException { - Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null); - HoodieDataBlock lastBlock = null; - while (reader.hasNext()) { - HoodieLogBlock block = reader.next(); - if (block instanceof HoodieDataBlock) { - lastBlock = (HoodieDataBlock) block; + try (Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null)) { + HoodieDataBlock lastBlock = null; + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieDataBlock) { + lastBlock = (HoodieDataBlock) block; + } } + return lastBlock != null ? new AvroSchemaConverter().convert(lastBlock.getSchema()) : null; } - reader.close(); - if (lastBlock != null) { - return new AvroSchemaConverter().convert(lastBlock.getSchema()); - } - return null; } public boolean isHasOperationField() { 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 d2501ee8dc15e..eb44769d9ff32 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 @@ -19,21 +19,11 @@ package org.apache.hudi.hadoop.utils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; -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.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; 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.hadoop.realtime.HoodieRealtimeBootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; @@ -41,14 +31,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.util.ArrayList; -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; - import static org.apache.hudi.TypeUtils.unsafeCast; public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { @@ -67,41 +49,6 @@ public static boolean doesBelongToIncrementalQuery(FileSplit s) { return false; } - // Return parquet file with a list of log files in the same file group. - public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { - Set partitionSet = new HashSet<>(partitionPaths); - // TODO(vc): Should we handle also non-hoodie splits here? - Map partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionSet); - - // Get all the base file and it's log files pairs in required partition paths. - List, List>> baseAndLogsList = new ArrayList<>(); - partitionSet.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 { - // Both commit and delta-commits are included - pick the latest completed one - Option latestCompletedInstant = - metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants().lastInstant(); - - Stream latestFileSlices = latestCompletedInstant - .map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp())) - .orElse(Stream.empty()); - - latestFileSlices.forEach(fileSlice -> { - List logFilePaths = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - baseAndLogsList.add(Pair.of(fileSlice.getBaseFile(), logFilePaths)); - }); - } catch (Exception e) { - throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath, e); - } - }); - return baseAndLogsList; - } - - /** * Add a field to the existing fields projected. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala new file mode 100644 index 0000000000000..da4e8d30e206f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyRelation.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.HoodieBaseRelation.createBaseFileReader +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, SQLContext} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.{BaseRelation, Filter} +import org.apache.spark.sql.types.StructType + +/** + * [[BaseRelation]] implementation only reading Base files of Hudi tables, essentially supporting following querying + * modes: + *

      + *
    • For COW tables: Snapshot
    • + *
    • For MOR tables: Read-optimized
    • + *
    + * + * NOTE: The reason this Relation is used in liue of Spark's default [[HadoopFsRelation]] is primarily due to the + * fact that it injects real partition's path as the value of the partition field, which Hudi ultimately persists + * as part of the record payload. In some cases, however, partition path might not necessarily be equal to the + * verbatim value of the partition path field (when custom [[KeyGenerator]] is used) therefore leading to incorrect + * partition field values being written + */ +class BaseFileOnlyRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, + optParams: Map[String, String], + userSchema: Option[StructType], + globPaths: Seq[Path]) + extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) with SparkAdapterSupport { + + override type FileSplit = HoodieBaseFileSplit + + protected override def composeRDD(fileSplits: Seq[HoodieBaseFileSplit], + partitionSchema: StructType, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Array[Filter]): HoodieUnsafeRDD = { + val baseFileReader = createBaseFileReader( + spark = sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = requiredSchema, + filters = filters, + options = optParams, + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) + ) + + new HoodieFileScanRDD(sparkSession, baseFileReader, fileSplits) + } + + protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[HoodieBaseFileSplit] = { + val partitions = listLatestBaseFiles(globPaths, partitionFilters, dataFilters) + val fileSplits = partitions.values.toSeq.flatMap { files => + files.flatMap { file => + // TODO move to adapter + // TODO fix, currently assuming parquet as underlying format + HoodieDataSourceHelper.splitFiles( + sparkSession = sparkSession, + file = file, + // TODO clarify why this is required + partitionValues = InternalRow.empty + ) + } + } + + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + + sparkAdapter.getFilePartitions(sparkSession, fileSplits, maxSplitBytes).map(HoodieBaseFileSplit.apply) + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala deleted file mode 100644 index 473bb2e2445bd..0000000000000 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala +++ /dev/null @@ -1,141 +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.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hudi.HoodieBaseRelation.createBaseFileReader -import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.hadoop.HoodieROTablePathFilter -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{HoodieCatalystExpressionUtils, SQLContext} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.sources.{BaseRelation, Filter} -import org.apache.spark.sql.types.StructType - -/** - * [[BaseRelation]] implementation only reading Base files of Hudi tables, essentially supporting following querying - * modes: - *
      - *
    • For COW tables: Snapshot
    • - *
    • For MOR tables: Read-optimized
    • - *
    - * - * NOTE: The reason this Relation is used in liue of Spark's default [[HadoopFsRelation]] is primarily due to the - * fact that it injects real partition's path as the value of the partition field, which Hudi ultimately persists - * as part of the record payload. In some cases, however, partition path might not necessarily be equal to the - * verbatim value of the partition path field (when custom [[KeyGenerator]] is used) therefore leading to incorrect - * partition field values being written - */ -class BaseFileOnlyViewRelation(sqlContext: SQLContext, - metaClient: HoodieTableMetaClient, - optParams: Map[String, String], - userSchema: Option[StructType], - globPaths: Seq[Path]) - extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) with SparkAdapterSupport { - - private val fileIndex = HoodieFileIndex(sparkSession, metaClient, userSchema, optParams, - FileStatusCache.getOrCreate(sqlContext.sparkSession)) - - override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { - // NOTE: In case list of requested columns doesn't contain the Primary Key one, we - // have to add it explicitly so that - // - Merging could be performed correctly - // - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]], - // Spark still fetches all the rows to execute the query correctly - // - // It's okay to return columns that have not been requested by the caller, as those nevertheless will be - // filtered out upstream - val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) - - val (requiredAvroSchema, requiredStructSchema) = - HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) - - val filterExpressions = convertToExpressions(filters) - val (partitionFilters, dataFilters) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates( - sparkSession, filterExpressions, partitionColumns) - - val filePartitions = getPartitions(partitionFilters, dataFilters) - - val partitionSchema = StructType(Nil) - val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) - val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) - - val baseFileReader = createBaseFileReader( - spark = sparkSession, - partitionSchema = partitionSchema, - tableSchema = tableSchema, - requiredSchema = requiredSchema, - filters = filters, - options = optParams, - // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it - // to configure Parquet reader appropriately - hadoopConf = new Configuration(conf) - ) - - new HoodieFileScanRDD(sparkSession, baseFileReader, filePartitions) - } - - private def getPartitions(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FilePartition] = { - val partitionDirectories = if (globPaths.isEmpty) { - val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, userSchema, optParams, - FileStatusCache.getOrCreate(sqlContext.sparkSession)) - hoodieFileIndex.listFiles(partitionFilters, dataFilters) - } else { - sqlContext.sparkContext.hadoopConfiguration.setClass( - "mapreduce.input.pathFilter.class", - classOf[HoodieROTablePathFilter], - classOf[org.apache.hadoop.fs.PathFilter]) - - val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sparkSession, globPaths) - inMemoryFileIndex.listFiles(partitionFilters, dataFilters) - } - - val partitions = partitionDirectories.flatMap { partition => - partition.files.flatMap { file => - // TODO move to adapter - // TODO fix, currently assuming parquet as underlying format - HoodieDataSourceHelper.splitFiles( - sparkSession = sparkSession, - file = file, - // TODO clarify why this is required - partitionValues = InternalRow.empty - ) - } - } - - val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes - - sparkAdapter.getFilePartitions(sparkSession, partitions, maxSplitBytes) - } - - private def convertToExpressions(filters: Array[Filter]): Array[Expression] = { - val catalystExpressions = HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema) - - val failedExprs = catalystExpressions.zipWithIndex.filter { case (opt, _) => opt.isEmpty } - if (failedExprs.nonEmpty) { - val failedFilters = failedExprs.map(p => filters(p._2)) - logWarning(s"Failed to convert Filters into Catalyst expressions (${failedFilters.map(_.toString)})") - } - - catalystExpressions.filter(_.isDefined).map(_.get).toArray - } -} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala index 7d558ec8cd10e..65dbdee127902 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -107,7 +107,7 @@ class DefaultSource extends RelationProvider case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) | (COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) | (MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) => - new BaseFileOnlyViewRelation(sqlContext, metaClient, parameters, userSchema, globPaths) + new BaseFileOnlyRelation(sqlContext, metaClient, parameters, userSchema, globPaths) case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) => new IncrementalRelation(sqlContext, parameters, userSchema, metaClient) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index e07b316d48db3..11778da63db31 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -20,22 +20,28 @@ package org.apache.hudi 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.hadoop.fs.{FileStatus, Path, PathFilter} import org.apache.hadoop.hbase.io.hfile.CacheConfig import org.apache.hadoop.mapred.JobConf -import org.apache.hudi.HoodieBaseRelation.isMetadataTable +import org.apache.hudi.HoodieBaseRelation.{getPartitionPath, isMetadataTable} +import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.common.config.SerializableConfiguration import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.hadoop.HoodieROTablePathFilter import org.apache.hudi.io.storage.HoodieHFileReader import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata} +import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression} +import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionDirectory, PartitionedFile} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.StructType @@ -44,6 +50,8 @@ import org.apache.spark.sql.{Row, SQLContext, SparkSession} import scala.collection.JavaConverters._ import scala.util.Try +trait HoodieFileSplit {} + case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String) case class HoodieTableState(recordKeyField: String, @@ -53,36 +61,33 @@ case class HoodieTableState(recordKeyField: String, * Hoodie BaseRelation which extends [[PrunedFilteredScan]]. */ abstract class HoodieBaseRelation(val sqlContext: SQLContext, - metaClient: HoodieTableMetaClient, - optParams: Map[String, String], + val metaClient: HoodieTableMetaClient, + val optParams: Map[String, String], userSchema: Option[StructType]) extends BaseRelation with PrunedFilteredScan with Logging { + type FileSplit <: HoodieFileSplit + + imbueConfigs(sqlContext) + protected val sparkSession: SparkSession = sqlContext.sparkSession protected lazy val conf: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) protected lazy val jobConf = new JobConf(conf) + protected lazy val tableConfig: HoodieTableConfig = metaClient.getTableConfig + + protected lazy val basePath: String = metaClient.getBasePath + // If meta fields are enabled, always prefer key from the meta field as opposed to user-specified one // NOTE: This is historical behavior which is preserved as is protected lazy val recordKeyField: String = - if (metaClient.getTableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD - else metaClient.getTableConfig.getRecordKeyFieldProp + if (tableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD + else tableConfig.getRecordKeyFieldProp protected lazy val preCombineFieldOpt: Option[String] = getPrecombineFieldProperty - /** - * @VisibleInTests - */ - lazy val mandatoryColumns: Seq[String] = { - if (isMetadataTable(metaClient)) { - Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE) - } else { - Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) - } - } - - protected lazy val specifiedQueryInstant: Option[String] = + protected lazy val specifiedQueryTimestamp: Option[String] = optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key) .map(HoodieSqlCommonUtils.formatQueryInstant) @@ -100,25 +105,49 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, protected val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema) - protected val partitionColumns: Array[String] = metaClient.getTableConfig.getPartitionFields.orElse(Array.empty) + protected val partitionColumns: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty) - protected def getPrecombineFieldProperty: Option[String] = - Option(metaClient.getTableConfig.getPreCombineField) - .orElse(optParams.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)) match { - // NOTE: This is required to compensate for cases when empty string is used to stub - // property value to avoid it being set with the default value - // TODO(HUDI-3456) cleanup - case Some(f) if !StringUtils.isNullOrEmpty(f) => Some(f) - case _ => None + /** + * NOTE: PLEASE READ THIS CAREFULLY + * + * Even though [[HoodieFileIndex]] initializes eagerly listing all of the files w/in the given Hudi table, + * this variable itself is _lazy_ (and have to stay that way) which guarantees that it's not initialized, until + * it's actually accessed + */ + protected lazy val fileIndex: HoodieFileIndex = + HoodieFileIndex(sparkSession, metaClient, Some(tableStructSchema), optParams, + FileStatusCache.getOrCreate(sparkSession)) + + /** + * @VisibleInTests + */ + lazy val mandatoryColumns: Seq[String] = { + if (isMetadataTable(metaClient)) { + Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE) + } else { + // TODO this is MOR table requirement, not necessary for COW + Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) } + } + + protected def timeline: HoodieTimeline = + // NOTE: We're including compaction here since it's not considering a "commit" operation + metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants + + protected def latestInstant: Option[HoodieInstant] = + toScalaOption(timeline.lastInstant()) + + protected def queryTimestamp: Option[String] = { + specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(i => i.getTimestamp)) + } override def schema: StructType = tableStructSchema /** * This method controls whether relation will be producing *
      - *
    • [[Row]], when it's being equal to true
    • - *
    • [[InternalRow]], when it's being equal to false
    • + *
    • [[Row]], when it's being equal to true
    • + *
    • [[InternalRow]], when it's being equal to false
    • *
    * * Returning [[InternalRow]] directly enables us to save on needless ser/de loop from [[InternalRow]] (being @@ -130,22 +159,129 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, * NOTE: DO NOT OVERRIDE THIS METHOD */ override final def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + // NOTE: In case list of requested columns doesn't contain the Primary Key one, we + // have to add it explicitly so that + // - Merging could be performed correctly + // - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]], + // Spark still fetches all the rows to execute the query correctly + // + // It's okay to return columns that have not been requested by the caller, as those nevertheless will be + // filtered out upstream + val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) + + val (requiredAvroSchema, requiredStructSchema) = + HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) + + val filterExpressions = convertToExpressions(filters) + val (partitionFilters, dataFilters) = filterExpressions.partition(isPartitionPredicate) + + val fileSplits = collectFileSplits(partitionFilters, dataFilters) + + val partitionSchema = StructType(Nil) + val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) + val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) + // Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]] // Please check [[needConversion]] scala-doc for more details - doBuildScan(requiredColumns, filters).asInstanceOf[RDD[Row]] + if (fileSplits.nonEmpty) + composeRDD(fileSplits, partitionSchema, tableSchema, requiredSchema, filters).asInstanceOf[RDD[Row]] + else + sparkSession.sparkContext.emptyRDD + } + + /** + * Composes RDD provided file splits to read from, table and partition schemas, data filters to be applied + * + * @param fileSplits file splits to be handled by the RDD + * @param partitionSchema target table's partition schema + * @param tableSchema target table's schema + * @param requiredSchema projected schema required by the reader + * @param filters data filters to be applied + * @return instance of RDD (implementing [[HoodieUnsafeRDD]]) + */ + protected def composeRDD(fileSplits: Seq[FileSplit], + partitionSchema: StructType, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Array[Filter]): HoodieUnsafeRDD + + /** + * Provided with partition and date filters collects target file splits to read records from, while + * performing pruning if necessary + * + * @param partitionFilters partition filters to be applied + * @param dataFilters data filters to be applied + * @return list of [[FileSplit]] to fetch records from + */ + protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FileSplit] + + protected def listLatestBaseFiles(globbedPaths: Seq[Path], partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Map[Path, Seq[FileStatus]] = { + val partitionDirs = if (globbedPaths.isEmpty) { + fileIndex.listFiles(partitionFilters, dataFilters) + } else { + val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, globbedPaths) + inMemoryFileIndex.listFiles(partitionFilters, dataFilters) + } + + val fsView = new HoodieTableFileSystemView(metaClient, timeline, partitionDirs.flatMap(_.files).toArray) + val latestBaseFiles = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus) + + latestBaseFiles.groupBy(getPartitionPath) } - protected def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] + protected def convertToExpressions(filters: Array[Filter]): Array[Expression] = { + val catalystExpressions = HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema) + + val failedExprs = catalystExpressions.zipWithIndex.filter { case (opt, _) => opt.isEmpty } + if (failedExprs.nonEmpty) { + val failedFilters = failedExprs.map(p => filters(p._2)) + logWarning(s"Failed to convert Filters into Catalyst expressions (${failedFilters.map(_.toString)})") + } + + catalystExpressions.filter(_.isDefined).map(_.get).toArray + } + + /** + * Checks whether given expression only references partition columns + * (and involves no sub-query) + */ + protected def isPartitionPredicate(condition: Expression): Boolean = { + // Validates that the provided names both resolve to the same entity + val resolvedNameEquals = sparkSession.sessionState.analyzer.resolver + + condition.references.forall { r => partitionColumns.exists(resolvedNameEquals(r.name, _)) } && + !SubqueryExpression.hasSubquery(condition) + } protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) requestedColumns ++ missing } + + private def getPrecombineFieldProperty: Option[String] = + Option(tableConfig.getPreCombineField) + .orElse(optParams.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key)) match { + // NOTE: This is required to compensate for cases when empty string is used to stub + // property value to avoid it being set with the default value + // TODO(HUDI-3456) cleanup + case Some(f) if !StringUtils.isNullOrEmpty(f) => Some(f) + case _ => None + } + + private def imbueConfigs(sqlContext: SQLContext): Unit = { + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true") + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true") + // TODO(HUDI-3639) vectorized reader has to be disabled to make sure MORIncrementalRelation is working properly + sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false") + } } object HoodieBaseRelation { - def isMetadataTable(metaClient: HoodieTableMetaClient) = + def getPartitionPath(fileStatus: FileStatus): Path = + fileStatus.getPath.getParent + + def isMetadataTable(metaClient: HoodieTableMetaClient): Boolean = HoodieTableMetadata.isMetadataTable(metaClient.getBasePath) /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala index dd90d724c6b61..00de2f756e912 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieBaseFile import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.exception.HoodieException +import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -157,7 +158,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, logInfo("Building file index..") val fileStatuses = if (globPaths.nonEmpty) { // Load files from the global paths if it has defined to be compatible with the original mode - val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(_sqlContext.sparkSession, globPaths) + val inMemoryFileIndex = HoodieInMemoryFileIndex.create(_sqlContext.sparkSession, globPaths) inMemoryFileIndex.allFiles() } else { // Load files by the HoodieFileIndex. HoodieFileIndex(sqlContext.sparkSession, metaClient, Some(schema), optParams, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala index 40299cfdcd6f1..0871487b5e8c6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala @@ -65,20 +65,6 @@ object HoodieDataSourceHelper extends PredicateHelper { } } - /** - * Extract the required schema from [[InternalRow]] - */ - def extractRequiredSchema( - iter: Iterator[InternalRow], - requiredSchema: StructType, - requiredFieldPos: Seq[Int]): Iterator[InternalRow] = { - val unsafeProjection = UnsafeProjection.create(requiredSchema) - val rows = iter.map { row => - unsafeProjection(createInternalRowWithSchema(row, requiredSchema, requiredFieldPos)) - } - rows - } - /** * Convert [[InternalRow]] to [[SpecificInternalRow]]. */ diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 6aa6fbb0eeaa9..de863203d6d5e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -348,7 +348,7 @@ object HoodieFileIndex extends Logging { } } catch { case NonFatal(e) => - logWarning("Fail to convert filters for TimestampBaseAvroKeyGenerator.") + logWarning("Fail to convert filters for TimestampBaseAvroKeyGenerator", e) partitionFilters } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala index aef6fc06222e7..cf68981d8318f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala @@ -24,12 +24,14 @@ import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, SchemaColumnConvertNotSupportedException} import org.apache.spark.{Partition, TaskContext} +case class HoodieBaseFileSplit(filePartition: FilePartition) extends HoodieFileSplit + /** * TODO eval if we actually need it */ class HoodieFileScanRDD(@transient private val sparkSession: SparkSession, readFunction: PartitionedFile => Iterator[InternalRow], - @transient fileSplits: Seq[FilePartition]) + @transient fileSplits: Seq[HoodieBaseFileSplit]) extends HoodieUnsafeRDD(sparkSession.sparkContext) { override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { @@ -77,5 +79,5 @@ class HoodieFileScanRDD(@transient private val sparkSession: SparkSession, iterator.asInstanceOf[Iterator[InternalRow]] } - override protected def getPartitions: Array[Partition] = fileSplits.toArray + override protected def getPartitions: Array[Partition] = fileSplits.map(_.filePartition).toArray } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 3a518da32b3dd..cc2915d605ff7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -56,7 +56,7 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, tableState: HoodieTableState, tableSchema: HoodieTableSchema, requiredSchema: HoodieTableSchema, - @transient fileSplits: List[HoodieMergeOnReadFileSplit]) + @transient fileSplits: Seq[HoodieMergeOnReadFileSplit]) extends HoodieUnsafeRDD(sc) { private val confBroadcast = sc.broadcast(new SerializableWritable(config)) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index 8308e3b7ee8ad..2517252d700fb 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -20,65 +20,134 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{GlobPattern, Path} import org.apache.hudi.HoodieBaseRelation.createBaseFileReader -import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.HoodieConversionUtils.toScalaOption +import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath +import org.apache.hudi.common.model.{FileSlice, HoodieRecord} import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} import org.apache.hudi.common.table.view.HoodieTableFileSystemView +import org.apache.hudi.common.util.StringUtils import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.{getCommitMetadata, getWritePartitionPaths, listAffectedFilesForCommits} -import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.immutable /** - * Experimental. - * Relation, that implements the Hoodie incremental view for Merge On Read table. - * + * @Experimental */ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, - val optParams: Map[String, String], - val userSchema: Option[StructType], - val metaClient: HoodieTableMetaClient) - extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { - - private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() - if (commitTimeline.empty()) { - throw new HoodieException("No instants to incrementally pull") + optParams: Map[String, String], + userSchema: Option[StructType], + metaClient: HoodieTableMetaClient) + extends MergeOnReadSnapshotRelation(sqlContext, optParams, userSchema, Seq(), metaClient) with HoodieIncrementalRelationTrait { + + override type FileSplit = HoodieMergeOnReadFileSplit + + override protected def timeline: HoodieTimeline = { + val startTimestamp = optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key) + val endTimestamp = optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getTimestamp) + super.timeline.findInstantsInRange(startTimestamp, endTimestamp) } - if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME.key)) { - throw new HoodieException(s"Specify the begin instant time to pull from using " + - s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME.key}") + + protected override def composeRDD(fileSplits: Seq[HoodieMergeOnReadFileSplit], + partitionSchema: StructType, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Array[Filter]): HoodieMergeOnReadRDD = { + val fullSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = tableSchema, + // This file-reader is used to read base file records, subsequently merging them with the records + // stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding + // applying any user-defined filtering _before_ we complete combining them w/ delta-log records (to make sure that + // we combine them correctly) + // + // The only filtering applicable here is the filtering to make sure we're only fetching records that + // fall into incremental span of the timeline being queried + filters = incrementalSpanRecordFilters, + options = optParams, + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) + ) + + val requiredSchemaParquetReader = createBaseFileReader( + spark = sqlContext.sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = requiredSchema, + filters = filters ++ incrementalSpanRecordFilters, + options = optParams, + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) + ) + + val hoodieTableState = HoodieTableState(HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt) + + // TODO(HUDI-3639) implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately + // filtered, since file-reader might not be capable to perform filtering + new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, + requiredSchemaParquetReader, hoodieTableState, tableSchema, requiredSchema, fileSplits) } - if (!metaClient.getTableConfig.populateMetaFields()) { - throw new HoodieException("Incremental queries are not supported when meta fields are disabled") + + override protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): List[HoodieMergeOnReadFileSplit] = { + if (includedCommits.isEmpty) { + List() + } else { + val latestCommit = includedCommits.last.getTimestamp + val commitsMetadata = includedCommits.map(getCommitMetadata(_, timeline)).asJava + + val modifiedFiles = listAffectedFilesForCommits(conf, new Path(metaClient.getBasePath), commitsMetadata) + val fsView = new HoodieTableFileSystemView(metaClient, timeline, modifiedFiles) + + val modifiedPartitions = getWritePartitionPaths(commitsMetadata) + + val fileSlices = modifiedPartitions.asScala.flatMap { relativePartitionPath => + fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, latestCommit).iterator().asScala + }.toSeq + + buildSplits(filterFileSlices(fileSlices, globPattern)) + } } - private val lastInstant = commitTimeline.lastInstant().get() - private val mergeType = optParams.getOrElse( - DataSourceReadOptions.REALTIME_MERGE.key, - DataSourceReadOptions.REALTIME_MERGE.defaultValue) + private def filterFileSlices(fileSlices: Seq[FileSlice], pathGlobPattern: String): Seq[FileSlice] = { + val filteredFileSlices = if (!StringUtils.isNullOrEmpty(pathGlobPattern)) { + val globMatcher = new GlobPattern("*" + pathGlobPattern) + fileSlices.filter(fileSlice => { + val path = toScalaOption(fileSlice.getBaseFile).map(_.getPath) + .orElse(toScalaOption(fileSlice.getLatestLogFile).map(_.getPath.toString)) + .get + globMatcher.matches(path) + }) + } else { + fileSlices + } + filteredFileSlices + } +} - private val commitsTimelineToReturn = commitTimeline.findInstantsInRange( - optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key), - optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, lastInstant.getTimestamp)) - logDebug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}") - private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList +trait HoodieIncrementalRelationTrait extends HoodieBaseRelation { - private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf) + // Validate this Incremental implementation is properly configured + validate() - private val fileIndex = if (commitsToReturn.isEmpty) List() else buildFileIndex() + protected lazy val includedCommits: immutable.Seq[HoodieInstant] = timeline.getInstants.iterator().asScala.toList // Record filters making sure that only records w/in the requested bounds are being fetched as part of the // scan collected by this relation - private lazy val incrementalSpanRecordsFilters: Seq[Filter] = { + protected lazy val incrementalSpanRecordFilters: Seq[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) + val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, includedCommits.head.getTimestamp) + val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, includedCommits.last.getTimestamp) + Seq(isNotNullFilter, largerThanFilter, lessThanFilter) } @@ -89,132 +158,23 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) } - override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { - if (fileIndex.isEmpty) { - sqlContext.sparkContext.emptyRDD[InternalRow] - } else { - logDebug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}") - logDebug(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 fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) - - val (requiredAvroSchema, requiredStructSchema) = - HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) - - val partitionSchema = StructType(Nil) - val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) - val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) - - val fullSchemaParquetReader = createBaseFileReader( - spark = sqlContext.sparkSession, - partitionSchema = partitionSchema, - tableSchema = tableSchema, - requiredSchema = tableSchema, - // This file-reader is used to read base file records, subsequently merging them with the records - // stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding - // applying any user-defined filtering _before_ we complete combining them w/ delta-log records (to make sure that - // we combine them correctly) - // - // The only filtering applicable here is the filtering to make sure we're only fetching records that - // fall into incremental span of the timeline being queried - filters = incrementalSpanRecordsFilters, - options = optParams, - // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it - // to configure Parquet reader appropriately - hadoopConf = new Configuration(conf) - ) - val requiredSchemaParquetReader = createBaseFileReader( - spark = sqlContext.sparkSession, - partitionSchema = partitionSchema, - tableSchema = tableSchema, - requiredSchema = requiredSchema, - filters = filters ++ incrementalSpanRecordsFilters, - options = optParams, - // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it - // to configure Parquet reader appropriately - hadoopConf = new Configuration(conf) - ) - - val hoodieTableState = HoodieTableState(HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt) - - // TODO implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately - // filtered, since file-reader might not be capable to perform filtering - new HoodieMergeOnReadRDD( - sqlContext.sparkContext, - jobConf, - fullSchemaParquetReader, - requiredSchemaParquetReader, - hoodieTableState, - tableSchema, - requiredSchema, - fileIndex - ) + protected def validate(): Unit = { + if (super.timeline.empty()) { + throw new HoodieException("No instants to incrementally pull") } - } - def buildFileIndex(): List[HoodieMergeOnReadFileSplit] = { - val metadataList = commitsToReturn.map(instant => getCommitMetadata(instant, commitsTimelineToReturn)) - val affectedFileStatus = listAffectedFilesForCommits(conf, new Path(metaClient.getBasePath), metadataList) - val fsView = new HoodieTableFileSystemView(metaClient, commitsTimelineToReturn, affectedFileStatus) - - // Iterate partitions to create splits - val fileGroups = getWritePartitionPaths(metadataList).flatMap(partitionPath => - fsView.getAllFileGroups(partitionPath).iterator() - ).toList - val latestCommit = fsView.getLastInstant.get.getTimestamp - if (log.isDebugEnabled) { - fileGroups.foreach(f => logDebug(s"current file group id: " + - s"${f.getFileGroupId} and file slices ${f.getLatestFileSlice.get.toString}")) + if (!this.optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME.key)) { + throw new HoodieException(s"Specify the begin instant time to pull from using " + + s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME.key}") } - // Filter files based on user defined glob pattern - val pathGlobPattern = optParams.getOrElse( - DataSourceReadOptions.INCR_PATH_GLOB.key, - DataSourceReadOptions.INCR_PATH_GLOB.defaultValue) - val filteredFileGroup = if (!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)) { - val globMatcher = new GlobPattern("*" + pathGlobPattern) - fileGroups.filter(fg => { - val latestFileSlice = fg.getLatestFileSlice.get - if (latestFileSlice.getBaseFile.isPresent) { - globMatcher.matches(latestFileSlice.getBaseFile.get.getPath) - } else { - globMatcher.matches(latestFileSlice.getLatestLogFile.get.getPath.toString) - } - }) - } else { - fileGroups + if (!this.tableConfig.populateMetaFields()) { + throw new HoodieException("Incremental queries are not supported when meta fields are disabled") } - - // 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 - val filePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath) - Option(PartitionedFile(InternalRow.empty, filePath, 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) - } - else { - Option.empty - } - - HoodieMergeOnReadFileSplit(partitionedFile, logPath, - latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) - }) } + + protected def globPattern: String = + optParams.getOrElse(DataSourceReadOptions.INCR_PATH_GLOB.key, DataSourceReadOptions.INCR_PATH_GLOB.defaultValue) + } + diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 6156054b4f45b..d2515e3297d0f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -21,16 +21,18 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieBaseRelation.createBaseFileReader -import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord} +import org.apache.hudi.HoodieConversionUtils.toScalaOption +import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath +import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath +import org.apache.hudi.common.model.{FileSlice, HoodieLogFile} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView -import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes -import org.apache.spark.rdd.RDD +import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType @@ -41,43 +43,28 @@ case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile], latestCommit: String, tablePath: String, maxCompactionMemoryInBytes: Long, - mergeType: String) + mergeType: String) extends HoodieFileSplit class MergeOnReadSnapshotRelation(sqlContext: SQLContext, optParams: Map[String, String], - val userSchema: Option[StructType], - val globPaths: Seq[Path], - val metaClient: HoodieTableMetaClient) + userSchema: Option[StructType], + globPaths: Seq[Path], + metaClient: HoodieTableMetaClient) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { + override type FileSplit = HoodieMergeOnReadFileSplit + private val mergeType = optParams.getOrElse( DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_MERGE.defaultValue) private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf) - override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { - log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}") - log.debug(s" buildScan filters = ${filters.mkString(",")}") - - // NOTE: In case list of requested columns doesn't contain the Primary Key one, we - // have to add it explicitly so that - // - Merging could be performed correctly - // - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]], - // Spark still fetches all the rows to execute the query correctly - // - // It's okay to return columns that have not been requested by the caller, as those nevertheless will be - // filtered out upstream - val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) - - val (requiredAvroSchema, requiredStructSchema) = - HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) - val fileIndex = buildFileIndex(filters) - - val partitionSchema = StructType(Nil) - val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) - val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) - + protected override def composeRDD(fileIndex: Seq[HoodieMergeOnReadFileSplit], + partitionSchema: StructType, + tableSchema: HoodieTableSchema, + requiredSchema: HoodieTableSchema, + filters: Array[Filter]): HoodieMergeOnReadRDD = { val fullSchemaParquetReader = createBaseFileReader( spark = sqlContext.sparkSession, partitionSchema = partitionSchema, @@ -93,6 +80,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, // to configure Parquet reader appropriately hadoopConf = new Configuration(conf) ) + val requiredSchemaParquetReader = createBaseFileReader( spark = sqlContext.sparkSession, partitionSchema = partitionSchema, @@ -111,90 +99,53 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, requiredSchemaParquetReader, tableState, tableSchema, requiredSchema, fileIndex) } - def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] = { - if (globPaths.nonEmpty) { - // Load files from the global paths if it has defined to be compatible with the original mode - val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths) - val fsView = new HoodieTableFileSystemView(metaClient, - // file-slice after pending compaction-requested instant-time is also considered valid - metaClient.getCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants, - inMemoryFileIndex.allFiles().toArray) - val partitionPaths = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent) + protected override def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): List[HoodieMergeOnReadFileSplit] = { + val convertedPartitionFilters = + HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient, partitionFilters) - - if (partitionPaths.isEmpty) { // If this an empty table, return an empty split list. - List.empty[HoodieMergeOnReadFileSplit] - } else { - val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants.lastInstant() - if (!lastInstant.isPresent) { // Return empty list if the table has no commit - List.empty - } else { - val queryInstant = specifiedQueryInstant.getOrElse(lastInstant.get().getTimestamp) - val baseAndLogsList = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, partitionPaths.asJava).asScala - val fileSplits = baseAndLogsList.map(kv => { - val baseFile = kv.getLeft - val logPaths = if (kv.getRight.isEmpty) Option.empty else Option(kv.getRight.asScala.toList) - - val baseDataPath = if (baseFile.isPresent) { - Some(PartitionedFile( - InternalRow.empty, - MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath), - 0, baseFile.get.getFileLen) - ) - } else { - None - } - HoodieMergeOnReadFileSplit(baseDataPath, logPaths, queryInstant, - metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) - }).toList - fileSplits - } - } + if (globPaths.isEmpty) { + val fileSlices = fileIndex.listFileSlices(convertedPartitionFilters) + buildSplits(fileSlices.values.flatten.toSeq) } else { - // Load files by the HoodieFileIndex. - val hoodieFileIndex = HoodieFileIndex(sqlContext.sparkSession, metaClient, - Some(tableStructSchema), optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession)) - - // Get partition filter and convert to catalyst expression - val partitionColumns = hoodieFileIndex.partitionSchema.fieldNames.toSet - val partitionFilters = filters.filter(f => f.references.forall(p => partitionColumns.contains(p))) - val partitionFilterExpression = - HoodieSparkUtils.convertToCatalystExpression(partitionFilters, tableStructSchema) - val convertedPartitionFilterExpression = - HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient, partitionFilterExpression.toSeq) - - // If convert success to catalyst expression, use the partition prune - val fileSlices = if (convertedPartitionFilterExpression.nonEmpty) { - hoodieFileIndex.listFileSlices(convertedPartitionFilterExpression) + // TODO refactor to avoid iterating over listed files multiple times + val partitions = listLatestBaseFiles(globPaths, convertedPartitionFilters, dataFilters) + val partitionPaths = partitions.keys.toSeq + if (partitionPaths.isEmpty || latestInstant.isEmpty) { + // If this an empty table OR it has no completed commits yet, return + List.empty[HoodieMergeOnReadFileSplit] } else { - hoodieFileIndex.listFileSlices(Seq.empty[Expression]) + val fileSlices = listFileSlices(partitionPaths) + buildSplits(fileSlices) } + } + } - if (fileSlices.isEmpty) { - // If this an empty table, return an empty split list. - List.empty[HoodieMergeOnReadFileSplit] - } else { - val fileSplits = fileSlices.values.flatten.map(fileSlice => { - val latestInstant = metaClient.getActiveTimeline.getCommitsTimeline - .filterCompletedInstants.lastInstant().get().getTimestamp - val queryInstant = specifiedQueryInstant.getOrElse(latestInstant) - - val partitionedFile = if (fileSlice.getBaseFile.isPresent) { - val baseFile = fileSlice.getBaseFile.get() - val baseFilePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.getFileStatus.getPath) - Option(PartitionedFile(InternalRow.empty, baseFilePath, 0, baseFile.getFileLen)) - } else { - Option.empty - } - - val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList - val logPathsOptional = if (logPaths.isEmpty) Option.empty else Option(logPaths) - - HoodieMergeOnReadFileSplit(partitionedFile, logPathsOptional, queryInstant, metaClient.getBasePath, - maxCompactionMemoryInBytes, mergeType) - }).toList - fileSplits + protected def buildSplits(fileSlices: Seq[FileSlice]): List[HoodieMergeOnReadFileSplit] = { + fileSlices.map { fileSlice => + val baseFile = toScalaOption(fileSlice.getBaseFile) + val logFiles = Option(fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala.toList) + + val partitionedBaseFile = baseFile.map { file => + val filePath = getFilePath(file.getFileStatus.getPath) + PartitionedFile(InternalRow.empty, filePath, 0, file.getFileLen) } + + HoodieMergeOnReadFileSplit(partitionedBaseFile, logFiles, queryTimestamp.get, + metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) + }.toList + } + + private def listFileSlices(partitionPaths: Seq[Path]): Seq[FileSlice] = { + // NOTE: It's critical for us to re-use [[InMemoryFileIndex]] to make sure we're leveraging + // [[FileStatusCache]] and avoid listing the whole table again + val inMemoryFileIndex = HoodieInMemoryFileIndex.create(sparkSession, partitionPaths) + val fsView = new HoodieTableFileSystemView(metaClient, timeline, inMemoryFileIndex.allFiles.toArray) + + val queryTimestamp = this.queryTimestamp.get + + partitionPaths.flatMap { partitionPath => + val relativePath = getRelativePartitionPath(new Path(basePath), partitionPath) + fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, queryTimestamp).iterator().asScala.toSeq } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 74b8e10c4415b..a06ffffe50e50 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters._ -import scala.language.implicitConversions /** * Implementation of the [[BaseHoodieTableFileIndex]] for Spark @@ -135,12 +134,14 @@ class SparkHoodieTableFileIndex(spark: SparkSession, * Fetch list of latest base files w/ corresponding log files, after performing * partition pruning * + * TODO unify w/ HoodieFileIndex#listFiles + * * @param partitionFilters partition column filters * @return mapping from string partition paths to its base/log files */ def listFileSlices(partitionFilters: Seq[Expression]): Map[String, Seq[FileSlice]] = { // Prune the partition path by the partition filters - val prunedPartitions = prunePartition(cachedAllInputFileSlices.asScala.keys.toSeq, partitionFilters) + val prunedPartitions = prunePartition(cachedAllInputFileSlices.keySet().asScala.toSeq, partitionFilters) prunedPartitions.map(partition => { (partition.path, cachedAllInputFileSlices.get(partition).asScala) }).toMap diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieHadoopFSUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieHadoopFSUtils.scala new file mode 100644 index 0000000000000..353d94a7c105f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieHadoopFSUtils.scala @@ -0,0 +1,370 @@ +/* + * 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.spark + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.viewfs.ViewFileSystem +import org.apache.hadoop.fs._ +import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.util.SerializableConfiguration + +import java.io.FileNotFoundException +import scala.collection.mutable + +/** + * NOTE: This method class is replica of HadoopFSUtils from Spark 3.2.1, with the following adjustments + * + * - Filtering out of the listed files is adjusted to include files starting w/ "." (to include Hoodie Delta Log + * files) + */ +object HoodieHadoopFSUtils extends Logging { + /** + * Lists a collection of paths recursively. Picks the listing strategy adaptively depending + * on the number of paths to list. + * + * This may only be called on the driver. + * + * @param sc Spark context used to run parallel listing. + * @param paths Input paths to list + * @param hadoopConf Hadoop configuration + * @param filter Path filter used to exclude leaf files from result + * @param ignoreMissingFiles Ignore missing files that happen during recursive listing + * (e.g., due to race conditions) + * @param ignoreLocality Whether to fetch data locality info when listing leaf files. If false, + * this will return `FileStatus` without `BlockLocation` info. + * @param parallelismThreshold The threshold to enable parallelism. If the number of input paths + * is smaller than this value, this will fallback to use + * sequential listing. + * @param parallelismMax The maximum parallelism for listing. If the number of input paths is + * larger than this value, parallelism will be throttled to this value + * to avoid generating too many tasks. + * @return for each input path, the set of discovered files for the path + */ + def parallelListLeafFiles(sc: SparkContext, + paths: Seq[Path], + hadoopConf: Configuration, + filter: PathFilter, + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + parallelismThreshold: Int, + parallelismMax: Int): Seq[(Path, Seq[FileStatus])] = { + parallelListLeafFilesInternal(sc, paths, hadoopConf, filter, isRootLevel = true, + ignoreMissingFiles, ignoreLocality, parallelismThreshold, parallelismMax) + } + + // scalastyle:off parameter.number + private def parallelListLeafFilesInternal(sc: SparkContext, + paths: Seq[Path], + hadoopConf: Configuration, + filter: PathFilter, + isRootLevel: Boolean, + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + parallelismThreshold: Int, + parallelismMax: Int): Seq[(Path, Seq[FileStatus])] = { + + // Short-circuits parallel listing when serial listing is likely to be faster. + if (paths.size <= parallelismThreshold) { + // scalastyle:off return + return paths.map { path => + val leafFiles = listLeafFiles( + path, + hadoopConf, + filter, + Some(sc), + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = isRootLevel, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax) + (path, leafFiles) + } + // scalastyle:on return + } + + logInfo(s"Listing leaf files and directories in parallel under ${paths.length} paths." + + s" The first several paths are: ${paths.take(10).mkString(", ")}.") + HiveCatalogMetrics.incrementParallelListingJobCount(1) + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + // Set the number of parallelism to prevent following file listing from generating many tasks + // in case of large #defaultParallelism. + val numParallelism = Math.min(paths.size, parallelismMax) + + val previousJobDescription = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) + val statusMap = try { + val description = paths.size match { + case 0 => + "Listing leaf files and directories 0 paths" + case 1 => + s"Listing leaf files and directories for 1 path:
    ${paths(0)}" + case s => + s"Listing leaf files and directories for $s paths:
    ${paths(0)}, ..." + } + sc.setJobDescription(description) + sc + .parallelize(serializedPaths, numParallelism) + .mapPartitions { pathStrings => + val hadoopConf = serializableConfiguration.value + pathStrings.map(new Path(_)).toSeq.map { path => + val leafFiles = listLeafFiles( + path = path, + hadoopConf = hadoopConf, + filter = filter, + contextOpt = None, // Can't execute parallel scans on workers + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = isRootLevel, + parallelismThreshold = Int.MaxValue, + parallelismMax = 0) + (path, leafFiles) + }.iterator + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } + + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) + }.collect() + } finally { + sc.setJobDescription(previousJobDescription) + } + + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) + } + (new Path(path), statuses) + } + } + // scalastyle:on parameter.number + + // scalastyle:off parameter.number + /** + * Lists a single filesystem path recursively. If a `SparkContext` object is specified, this + * function may launch Spark jobs to parallelize listing based on `parallelismThreshold`. + * + * If sessionOpt is None, this may be called on executors. + * + * @return all children of path that match the specified filter. + */ + private def listLeafFiles(path: Path, + hadoopConf: Configuration, + filter: PathFilter, + contextOpt: Option[SparkContext], + ignoreMissingFiles: Boolean, + ignoreLocality: Boolean, + isRootPath: Boolean, + parallelismThreshold: Int, + parallelismMax: Int): Seq[FileStatus] = { + + logTrace(s"Listing $path") + val fs = path.getFileSystem(hadoopConf) + + // Note that statuses only include FileStatus for the files and dirs directly under path, + // and does not include anything else recursively. + val statuses: Array[FileStatus] = try { + fs match { + // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode + // to retrieve the file status with the file block location. The reason to still fallback + // to listStatus is because the default implementation would potentially throw a + // FileNotFoundException which is better handled by doing the lookups manually below. + case (_: DistributedFileSystem | _: ViewFileSystem) if !ignoreLocality => + val remoteIter = fs.listLocatedStatus(path) + new Iterator[LocatedFileStatus]() { + def next(): LocatedFileStatus = remoteIter.next + + def hasNext(): Boolean = remoteIter.hasNext + }.toArray + case _ => fs.listStatus(path) + } + } catch { + // If we are listing a root path for SQL (e.g. a top level directory of a table), we need to + // ignore FileNotFoundExceptions during this root level of the listing because + // + // (a) certain code paths might construct an InMemoryFileIndex with root paths that + // might not exist (i.e. not all callers are guaranteed to have checked + // path existence prior to constructing InMemoryFileIndex) and, + // (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break + // existing behavior and break the ability drop SessionCatalog tables when tables' + // root directories have been deleted (which breaks a number of Spark's own tests). + // + // If we are NOT listing a root path then a FileNotFoundException here means that the + // directory was present in a previous level of file listing but is absent in this + // listing, likely indicating a race condition (e.g. concurrent table overwrite or S3 + // list inconsistency). + // + // The trade-off in supporting existing behaviors / use-cases is that we won't be + // able to detect race conditions involving root paths being deleted during + // InMemoryFileIndex construction. However, it's still a net improvement to detect and + // fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion. + case _: FileNotFoundException if isRootPath || ignoreMissingFiles => + logWarning(s"The directory $path was not found. Was it deleted very recently?") + Array.empty[FileStatus] + } + + val filteredStatuses = + statuses.filterNot(status => shouldFilterOutPathName(status.getPath.getName)) + + val allLeafStatuses = { + val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) + val nestedFiles: Seq[FileStatus] = contextOpt match { + case Some(context) if dirs.size > parallelismThreshold => + parallelListLeafFilesInternal( + context, + dirs.map(_.getPath), + hadoopConf = hadoopConf, + filter = filter, + isRootLevel = false, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax + ).flatMap(_._2) + case _ => + dirs.flatMap { dir => + listLeafFiles( + path = dir.getPath, + hadoopConf = hadoopConf, + filter = filter, + contextOpt = contextOpt, + ignoreMissingFiles = ignoreMissingFiles, + ignoreLocality = ignoreLocality, + isRootPath = false, + parallelismThreshold = parallelismThreshold, + parallelismMax = parallelismMax) + } + } + val allFiles = topLevelFiles ++ nestedFiles + if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles + } + + val missingFiles = mutable.ArrayBuffer.empty[String] + val resolvedLeafStatuses = allLeafStatuses.flatMap { + case f: LocatedFileStatus => + Some(f) + + // NOTE: + // + // - Although S3/S3A/S3N file system can be quite slow for remote file metadata + // operations, calling `getFileBlockLocations` does no harm here since these file system + // implementations don't actually issue RPC for this method. + // + // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not + // be a big deal since we always use to `parallelListLeafFiles` when the number of + // paths exceeds threshold. + case f if !ignoreLocality => + // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), + // which is very slow on some file system (RawLocalFileSystem, which is launch a + // subprocess and parse the stdout). + try { + val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc => + // Store BlockLocation objects to consume less memory + if (loc.getClass == classOf[BlockLocation]) { + loc + } else { + new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength) + } + } + val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, + f.getModificationTime, 0, null, null, null, null, f.getPath, locations) + if (f.isSymlink) { + lfs.setSymlink(f.getSymlink) + } + Some(lfs) + } catch { + case _: FileNotFoundException if ignoreMissingFiles => + missingFiles += f.getPath.toString + None + } + + case f => Some(f) + } + + if (missingFiles.nonEmpty) { + logWarning( + s"the following files were missing during file scan:\n ${missingFiles.mkString("\n ")}") + } + + resolvedLeafStatuses + } + // scalastyle:on parameter.number + + /** A serializable variant of HDFS's BlockLocation. This is required by Hadoop 2.7. */ + private case class SerializableBlockLocation(names: Array[String], + hosts: Array[String], + offset: Long, + length: Long) + + /** A serializable variant of HDFS's FileStatus. This is required by Hadoop 2.7. */ + private case class SerializableFileStatus(path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long, + blockLocations: Array[SerializableBlockLocation]) + + /** Checks if we should filter out this path name. */ + def shouldFilterOutPathName(pathName: String): Boolean = { + // We filter follow paths: + // 1. everything that starts with _ and ., except _common_metadata and _metadata + // because Parquet needs to find those metadata files from leaf files returned by this method. + // We should refactor this logic to not mix metadata files with data files. + // 2. everything that ends with `._COPYING_`, because this is a intermediate state of file. we + // should skip this file in case of double reading. + val exclude = (pathName.startsWith("_") && !pathName.contains("=")) || pathName.endsWith("._COPYING_") + val include = pathName.startsWith("_common_metadata") || pathName.startsWith("_metadata") + exclude && !include + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala new file mode 100644 index 0000000000000..8ad78af7f3267 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/execution/datasources/HoodieInMemoryFileIndex.scala @@ -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.spark.execution.datasources + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path, PathFilter} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.HoodieHadoopFSUtils +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.types.StructType + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +class HoodieInMemoryFileIndex(sparkSession: SparkSession, + rootPathsSpecified: Seq[Path], + parameters: Map[String, String], + userSpecifiedSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache) + extends InMemoryFileIndex(sparkSession, rootPathsSpecified, parameters, userSpecifiedSchema, fileStatusCache) { + + /** + * List leaf files of given paths. This method will submit a Spark job to do parallel + * listing whenever there is a path having more files than the parallel partition discovery threshold. + * + * This is publicly visible for testing. + * + * NOTE: This method replicates the one it overrides, however it uses custom method to run parallel + * listing that accepts files starting with "." + */ + override def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + val startTime = System.nanoTime() + val output = mutable.LinkedHashSet[FileStatus]() + val pathsToFetch = mutable.ArrayBuffer[Path]() + for (path <- paths) { + fileStatusCache.getLeafFiles(path) match { + case Some(files) => + HiveCatalogMetrics.incrementFileCacheHits(files.length) + output ++= files + case None => + pathsToFetch += path + } + () // for some reasons scalac 2.12 needs this; return type doesn't matter + } + val filter = FileInputFormat.getInputPathFilter(new JobConf(hadoopConf, this.getClass)) + val discovered = bulkListLeafFiles(sparkSession, pathsToFetch, filter, hadoopConf) + + discovered.foreach { case (path, leafFiles) => + HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) + fileStatusCache.putLeafFiles(path, leafFiles.toArray) + output ++= leafFiles + } + + logInfo(s"It took ${(System.nanoTime() - startTime) / (1000 * 1000)} ms to list leaf files" + + s" for ${paths.length} paths.") + + output + } + + protected def bulkListLeafFiles(sparkSession: SparkSession, paths: ArrayBuffer[Path], filter: PathFilter, hadoopConf: Configuration): Seq[(Path, Seq[FileStatus])] = { + HoodieHadoopFSUtils.parallelListLeafFiles( + sc = sparkSession.sparkContext, + paths = paths, + hadoopConf = hadoopConf, + filter = new PathFilterWrapper(filter), + ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles, + // NOTE: We're disabling fetching Block Info to speed up file listing + ignoreLocality = true, + parallelismThreshold = sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold, + parallelismMax = sparkSession.sessionState.conf.parallelPartitionDiscoveryParallelism) + } +} + +object HoodieInMemoryFileIndex { + def create(sparkSession: SparkSession, globbedPaths: Seq[Path]): HoodieInMemoryFileIndex = { + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + new HoodieInMemoryFileIndex(sparkSession, globbedPaths, Map(), Option.empty, fileStatusCache) + } +} + +private class PathFilterWrapper(val filter: PathFilter) extends PathFilter with Serializable { + override def accept(path: Path): Boolean = { + (filter == null || filter.accept(path)) && !HoodieHadoopFSUtils.shouldFilterOutPathName(path.getName) + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/TestHoodieInMemoryFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/TestHoodieInMemoryFileIndex.scala new file mode 100644 index 0000000000000..8e7f6bf14b7e5 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/scala/org/apache/spark/execution/datasources/TestHoodieInMemoryFileIndex.scala @@ -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.spark.execution.datasources + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.io.TempDir + +import java.io.File +import java.nio.file.Paths + +class TestHoodieInMemoryFileIndex { + + @Test + def testCreateInMemoryIndex(@TempDir tempDir: File): Unit = { + val spark = SparkSession.builder + .appName("Hoodie Datasource test") + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .getOrCreate + + val folders: Seq[Path] = Seq( + new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri), + new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri) + ) + + val files: Seq[Path] = Seq( + new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri), + new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri), + new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri), + new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri) + ) + + folders.foreach(folder => new File(folder.toUri).mkdir()) + files.foreach(file => new File(file.toUri).createNewFile()) + + val index = HoodieInMemoryFileIndex.create(spark, Seq(folders(0), folders(1))) + val indexedFilePaths = index.allFiles().map(fs => fs.getPath) + assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString)) + spark.stop() + } + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala index 9f00b5dcdf64f..39ee6e0fa7187 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkUtils.scala @@ -88,35 +88,6 @@ class TestHoodieSparkUtils { .sortWith(_.toString < _.toString), globbedPaths.sortWith(_.toString < _.toString)) } - @Test - def testCreateInMemoryIndex(@TempDir tempDir: File): Unit = { - val spark = SparkSession.builder - .appName("Hoodie Datasource test") - .master("local[2]") - .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .getOrCreate - - val folders: Seq[Path] = Seq( - new Path(Paths.get(tempDir.getAbsolutePath, "folder1").toUri), - new Path(Paths.get(tempDir.getAbsolutePath, "folder2").toUri) - ) - - val files: Seq[Path] = Seq( - new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file1").toUri), - new Path(Paths.get(tempDir.getAbsolutePath, "folder1", "file2").toUri), - new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file3").toUri), - new Path(Paths.get(tempDir.getAbsolutePath, "folder2", "file4").toUri) - ) - - folders.foreach(folder => new File(folder.toUri).mkdir()) - files.foreach(file => new File(file.toUri).createNewFile()) - - val index = HoodieSparkUtils.createInMemoryFileIndex(spark, Seq(folders(0), folders(1))) - val indexedFilePaths = index.allFiles().map(fs => fs.getPath) - assertEquals(files.sortWith(_.toString < _.toString), indexedFilePaths.sortWith(_.toString < _.toString)) - spark.stop() - } - @Test def testCreateRddSchemaEvol(): Unit = { val spark = SparkSession.builder diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala index a963081749455..ca5f79191a729 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -112,9 +112,9 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with val fullColumnsReadStats: Array[(String, Long)] = if (HoodieSparkUtils.isSpark3) Array( - ("rider", 14665), - ("rider,driver", 14665), - ("rider,driver,tip_history", 14665)) + ("rider", 14166), + ("rider,driver", 14166), + ("rider,driver,tip_history", 14166)) else if (HoodieSparkUtils.isSpark2) // TODO re-enable tests (these tests are very unstable currently) Array( @@ -163,11 +163,29 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with else fail("Only Spark 3 and Spark 2 are currently supported") + // Stats for the reads fetching _all_ columns (currently for MOR to be able to merge + // records properly full row has to be fetched; note, how amount of bytes read + // is invariant of the # of columns) + val fullColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 14166), + ("rider,driver", 14166), + ("rider,driver,tip_history", 14166)) + else if (HoodieSparkUtils.isSpark2) + // TODO re-enable tests (these tests are very unstable currently) + Array( + ("rider", -1), + ("rider,driver", -1), + ("rider,driver,tip_history", -1)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + // Test MOR / Snapshot / Skip-merge runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, projectedColumnsReadStats) // Test MOR / Snapshot / Payload-combine - runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, projectedColumnsReadStats) + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, fullColumnsReadStats) // Test MOR / Read Optimized runTest(tableState, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, "null", projectedColumnsReadStats) @@ -209,9 +227,9 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with val fullColumnsReadStats: Array[(String, Long)] = if (HoodieSparkUtils.isSpark3) Array( - ("rider", 19683), - ("rider,driver", 19683), - ("rider,driver,tip_history", 19683)) + ("rider", 19684), + ("rider,driver", 19684), + ("rider,driver,tip_history", 19684)) else if (HoodieSparkUtils.isSpark2) // TODO re-enable tests (these tests are very unstable currently) Array( From 1b6e201160ef18b0c7d86d1cafd104609489dad7 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Sat, 19 Mar 2022 21:54:13 -0700 Subject: [PATCH 26/33] [HUDI-3663] Fixing Column Stats index to properly handle first Data Table commit (#5070) * Fixed metadata conversion util to extract schema from `HoodieCommitMetadata` * Fixed failure to fetch columns to index in empty table * Abort indexing seq in case there are no columns to index * Fallback to index at least primary key columns, in case no writer schema could be obtained to index all columns * Fixed `getRecordFields` incorrectly ignoring default value * Make sure Hudi metadata fields are also indexed --- .../client/functional/TestHoodieIndex.java | 19 ++- .../hudi/common/table/HoodieTableConfig.java | 8 +- .../org/apache/hudi/common/util/Option.java | 16 +- .../metadata/HoodieTableMetadataUtil.java | 142 +++++++++++------- 4 files changed, 119 insertions(+), 66 deletions(-) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index 876a5d8dea12f..8c27e488dc782 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -72,6 +72,7 @@ import java.util.Properties; import java.util.Random; import java.util.UUID; +import java.util.stream.Collectors; import java.util.stream.Stream; import scala.Tuple2; @@ -258,19 +259,23 @@ public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, bool metaClient = HoodieTableMetaClient.reload(metaClient); // Insert 200 records - JavaRDD writeStatues = writeClient.upsert(writeRecords, newCommitTime); - Assertions.assertNoWriteErrors(writeStatues.collect()); - List fileIds = writeStatues.map(WriteStatus::getFileId).collect(); - // commit this upsert - writeClient.commit(newCommitTime, writeStatues); + JavaRDD writeStatusesRDD = writeClient.upsert(writeRecords, newCommitTime); + // NOTE: This will trigger an actual write + List writeStatuses = writeStatusesRDD.collect(); + Assertions.assertNoWriteErrors(writeStatuses); + // Commit + writeClient.commit(newCommitTime, jsc.parallelize(writeStatuses)); + + List fileIds = writeStatuses.stream().map(WriteStatus::getFileId).collect(Collectors.toList()); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); // Now tagLocation for these records, hbaseIndex should tag them JavaRDD javaRDD = tagLocation(index, writeRecords, hoodieTable); - assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords); + assertEquals(totalRecords, javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size()); // check tagged records are tagged with correct fileIds - assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0); + assertEquals(0, javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size()); List taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect(); Map recordKeyToPartitionPathMap = new HashMap(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 077cc81ef1a60..bc8a5c443d39f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -461,11 +461,9 @@ public String getPreCombineField() { } public Option getRecordKeyFields() { - if (contains(RECORDKEY_FIELDS)) { - return Option.of(Arrays.stream(getString(RECORDKEY_FIELDS).split(",")) - .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {})); - } - return Option.empty(); + String keyFieldsValue = getStringOrDefault(RECORDKEY_FIELDS, HoodieRecord.RECORD_KEY_METADATA_FIELD); + return Option.of(Arrays.stream(keyFieldsValue.split(",")) + .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {})); } public Option getPartitionFields() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/Option.java b/hudi-common/src/main/java/org/apache/hudi/common/util/Option.java index 193bf5315fd01..3d4bfcb6c84e4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/Option.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/Option.java @@ -34,7 +34,7 @@ public final class Option implements Serializable { private static final long serialVersionUID = 0L; - private static final Option NULL_VAL = new Option<>(); + private static final Option EMPTY = new Option<>(); private final T val; @@ -67,8 +67,9 @@ private Option(T val) { this.val = val; } + @SuppressWarnings("unchecked") public static Option empty() { - return (Option) NULL_VAL; + return (Option) EMPTY; } public static Option of(T value) { @@ -108,6 +109,17 @@ public Option map(Function mapper) { } } + public Option flatMap(Function> mapper) { + if (null == mapper) { + throw new NullPointerException("mapper should not be null"); + } + if (!isPresent()) { + return empty(); + } else { + return Objects.requireNonNull(mapper.apply(val)); + } + } + /** * Returns this {@link Option} if not empty, otherwise evaluates the provided supplier * and returns the alternative 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 index f0388cca14130..4390e8766c6aa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -67,6 +68,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -78,6 +80,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldValAsString; import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.COLUMN_RANGE_MERGE_FUNCTION; import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MAX; @@ -86,6 +89,7 @@ import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_SIZE; import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_UNCOMPRESSED_SIZE; import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.VALUE_COUNT; +import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; @@ -379,15 +383,24 @@ public static HoodieData convertMetadataToColumnStatsRecords(Hoodi deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry))); }); - final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); - final int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); - HoodieData> deleteFileListRDD = engineContext.parallelize(deleteFileList, parallelism); - return deleteFileListRDD.flatMap(deleteFileInfoPair -> { - if (deleteFileInfoPair.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { - return getColumnStats(deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), recordsGenerationParams.getDataMetaClient(), columnsToIndex, true).iterator(); - } - return Collections.emptyListIterator(); - }); + HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient(); + + List columnsToIndex = getColumnsToIndex(recordsGenerationParams, + dataTableMetaClient.getTableConfig(), tryResolveSchemaForTable(dataTableMetaClient)); + + if (columnsToIndex.isEmpty()) { + // In case there are no columns to index, bail + return engineContext.emptyHoodieData(); + } + + int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + return engineContext.parallelize(deleteFileList, parallelism) + .flatMap(deleteFileInfoPair -> { + if (deleteFileInfoPair.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + return getColumnStats(deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), dataTableMetaClient, columnsToIndex, true).iterator(); + } + return Collections.emptyListIterator(); + }); } /** @@ -698,7 +711,15 @@ public static HoodieData convertFilesToColumnStatsRecords(HoodieEn Map> partitionToAppendedFiles, MetadataRecordsGenerationParams recordsGenerationParams) { HoodieData allRecordsRDD = engineContext.emptyHoodieData(); - final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); + HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient(); + + final List columnsToIndex = getColumnsToIndex(recordsGenerationParams, + dataTableMetaClient.getTableConfig(), tryResolveSchemaForTable(dataTableMetaClient)); + + if (columnsToIndex.isEmpty()) { + // In case there are no columns to index, bail + return engineContext.emptyHoodieData(); + } final List>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet() .stream().map(e -> Pair.of(e.getKey(), e.getValue())).collect(Collectors.toList()); @@ -712,7 +733,7 @@ public static HoodieData convertFilesToColumnStatsRecords(HoodieEn return deletedFileList.stream().flatMap(deletedFile -> { final String filePathWithPartition = partitionName + "/" + deletedFile; - return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), columnsToIndex, true); + return getColumnStats(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, true); }).iterator(); }); allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD); @@ -733,7 +754,7 @@ public static HoodieData convertFilesToColumnStatsRecords(HoodieEn return Stream.empty(); } final String filePathWithPartition = partitionName + "/" + appendedFileNameLengthEntry.getKey(); - return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), columnsToIndex, false); + return getColumnStats(partition, filePathWithPartition, dataTableMetaClient, columnsToIndex, false); }).iterator(); }); @@ -838,55 +859,59 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta public static HoodieData convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, HoodieEngineContext engineContext, MetadataRecordsGenerationParams recordsGenerationParams) { - try { - List allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() - .flatMap(entry -> entry.stream()).collect(Collectors.toList()); - return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, allWriteStats, recordsGenerationParams); - } catch (Exception e) { - throw new HoodieException("Failed to generate column stats records for metadata table ", e); - } - } + List allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() + .flatMap(Collection::stream).collect(Collectors.toList()); - /** - * Create column stats from write status. - * - * @param engineContext - Engine context - * @param allWriteStats - Write status to convert - * @param recordsGenerationParams - Parameters for columns stats record generation - */ - public static HoodieData createColumnStatsFromWriteStats(HoodieEngineContext engineContext, - List allWriteStats, - MetadataRecordsGenerationParams recordsGenerationParams) { if (allWriteStats.isEmpty()) { return engineContext.emptyHoodieData(); } - final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); - final int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); - HoodieData allWriteStatsRDD = engineContext.parallelize(allWriteStats, parallelism); - return allWriteStatsRDD.flatMap(writeStat -> translateWriteStatToColumnStats(writeStat, recordsGenerationParams.getDataMetaClient(), columnsToIndex).iterator()); + + try { + Option writerSchema = + Option.ofNullable(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY)) + .flatMap(writerSchemaStr -> + isNullOrEmpty(writerSchemaStr) + ? Option.empty() + : Option.of(new Schema.Parser().parse(writerSchemaStr))); + + HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient(); + HoodieTableConfig tableConfig = dataTableMetaClient.getTableConfig(); + + // NOTE: Writer schema added to commit metadata will not contain Hudi's metadata fields + Option tableSchema = writerSchema.map(schema -> + tableConfig.populateMetaFields() ? addMetadataFields(schema) : schema); + + List columnsToIndex = getColumnsToIndex(recordsGenerationParams, + tableConfig, tableSchema); + + if (columnsToIndex.isEmpty()) { + // In case there are no columns to index, bail + return engineContext.emptyHoodieData(); + } + + int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + return engineContext.parallelize(allWriteStats, parallelism) + .flatMap(writeStat -> + translateWriteStatToColumnStats(writeStat, dataTableMetaClient, columnsToIndex).iterator()); + } catch (Exception e) { + throw new HoodieException("Failed to generate column stats records for metadata table", e); + } } /** * Get the latest columns for the table for column stats indexing. - * - * @param datasetMetaClient - Data table meta client - * @param isMetaIndexColumnStatsForAllColumns - Is column stats indexing enabled for all columns */ - private static List getColumnsToIndex(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) { - if (!isMetaIndexColumnStatsForAllColumns - || datasetMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() < 1) { - return Arrays.asList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp().split(",")); + private static List getColumnsToIndex(MetadataRecordsGenerationParams recordsGenParams, + HoodieTableConfig tableConfig, + Option writerSchemaOpt) { + if (recordsGenParams.isAllColumnStatsIndexEnabled() && writerSchemaOpt.isPresent()) { + return writerSchemaOpt.get().getFields().stream() + .map(Schema.Field::name).collect(Collectors.toList()); } - TableSchemaResolver schemaResolver = new TableSchemaResolver(datasetMetaClient); - // consider nested fields as well. if column stats is enabled only for a subset of columns, - // directly use them instead of all columns from the latest table schema - try { - return schemaResolver.getTableAvroSchema().getFields().stream() - .map(entry -> entry.name()).collect(Collectors.toList()); - } catch (Exception e) { - throw new HoodieException("Failed to get latest columns for " + datasetMetaClient.getBasePath()); - } + // In case no writer schema could be obtained we fall back to only index primary key + // columns + return Arrays.asList(tableConfig.getRecordKeyFields().get()); } public static HoodieMetadataColumnStats mergeColumnStats(HoodieMetadataColumnStats oldColumnStats, HoodieMetadataColumnStats newColumnStats) { @@ -914,7 +939,7 @@ public static Stream translateWriteStatToColumnStats(HoodieWriteSt List> columnRangeMetadataList = new ArrayList<>(columnRangeMap.values()); return HoodieMetadataPayload.createColumnStatsRecords(writeStat.getPartitionPath(), columnRangeMetadataList, false); } - return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex,false); + return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex, false); } private static Stream getColumnStats(final String partitionPath, final String filePathWithPartition, @@ -1023,7 +1048,7 @@ public static void aggregateColumnStats(IndexedRecord record, Schema schema, columnStats.put(TOTAL_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()) + fieldSize); columnStats.put(TOTAL_UNCOMPRESSED_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString()) + fieldSize); - if (!StringUtils.isNullOrEmpty(fieldVal)) { + if (!isNullOrEmpty(fieldVal)) { // set the min value of the field if (!columnStats.containsKey(MIN)) { columnStats.put(MIN, fieldVal); @@ -1043,4 +1068,17 @@ public static void aggregateColumnStats(IndexedRecord record, Schema schema, } }); } + + private static Option tryResolveSchemaForTable(HoodieTableMetaClient dataTableMetaClient) { + if (dataTableMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() == 0) { + return Option.empty(); + } + + TableSchemaResolver schemaResolver = new TableSchemaResolver(dataTableMetaClient); + try { + return Option.of(schemaResolver.getTableAvroSchema()); + } catch (Exception e) { + throw new HoodieException("Failed to get latest columns for " + dataTableMetaClient.getBasePath(), e); + } + } } From 15d1c186255ced3a8e08020ed68de9cd380dc250 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Sun, 20 Mar 2022 15:58:30 -0700 Subject: [PATCH 27/33] [MINOR] Remove flaky assert in TestInLineFileSystem (#5069) --- .../org/apache/hudi/common/fs/inline/TestInLineFileSystem.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java index 92f83aad7fd7e..88bd35ef4b536 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystem.java @@ -369,7 +369,6 @@ private Path getRandomInlinePath() { private void verifyFileStatus(FileStatus expected, Path inlinePath, long expectedLength, FileStatus actual) { assertEquals(inlinePath, actual.getPath()); assertEquals(expectedLength, actual.getLen()); - assertEquals(expected.getAccessTime(), actual.getAccessTime()); assertEquals(expected.getBlockSize(), actual.getBlockSize()); assertEquals(expected.getGroup(), actual.getGroup()); assertEquals(expected.getModificationTime(), actual.getModificationTime()); From 799c78e6888ef6a375c6779c3cfe7067756d4be9 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Mon, 21 Mar 2022 10:34:50 +0800 Subject: [PATCH 28/33] [HUDI-3665] Support flink multiple versions (#5072) --- azure-pipelines.yml | 8 +- hudi-client/hudi-flink-client/pom.xml | 4 +- hudi-flink-datasource/hudi-flink/pom.xml | 364 ++++++++++++++++++ .../hudi/configuration/FlinkOptions.java | 4 +- .../hudi/configuration/OptionsResolver.java | 0 .../hudi/schema/FilebasedSchemaProvider.java | 0 .../apache/hudi/schema/SchemaProvider.java | 0 .../hudi/schema/SchemaRegistryProvider.java | 0 .../hudi/sink/BucketStreamWriteFunction.java | 13 +- .../hudi/sink/BucketStreamWriteOperator.java | 3 +- .../org/apache/hudi/sink/CleanFunction.java | 0 .../apache/hudi/sink/StreamWriteFunction.java | 0 .../apache/hudi/sink/StreamWriteOperator.java | 0 .../sink/StreamWriteOperatorCoordinator.java | 0 .../hudi/sink/append/AppendWriteFunction.java | 0 .../hudi/sink/append/AppendWriteOperator.java | 0 .../sink/bootstrap/BootstrapOperator.java | 6 +- .../hudi/sink/bootstrap/IndexRecord.java | 0 .../aggregate/BootstrapAccumulator.java | 0 .../aggregate/BootstrapAggFunction.java | 0 .../batch/BatchBootstrapOperator.java | 0 .../sink/bulk/BulkInsertWriteFunction.java | 0 .../sink/bulk/BulkInsertWriteOperator.java | 0 .../sink/bulk/BulkInsertWriterHelper.java | 0 .../apache/hudi/sink/bulk/RowDataKeyGen.java | 0 .../hudi/sink/bulk/sort/SortOperator.java | 0 .../hudi/sink/bulk/sort/SortOperatorGen.java | 0 .../common/AbstractStreamWriteFunction.java | 0 .../sink/common/AbstractWriteFunction.java | 1 + .../sink/common/AbstractWriteOperator.java | 0 .../sink/common/WriteOperatorFactory.java | 0 .../hudi/sink/compact/CompactFunction.java | 0 .../sink/compact/CompactionCommitEvent.java | 0 .../sink/compact/CompactionCommitSink.java | 0 .../sink/compact/CompactionPlanEvent.java | 0 .../sink/compact/CompactionPlanOperator.java | 0 .../compact/CompactionPlanSourceFunction.java | 0 .../sink/compact/FlinkCompactionConfig.java | 0 .../sink/compact/HoodieFlinkCompactor.java | 0 .../hudi/sink/event/CommitAckEvent.java | 0 .../hudi/sink/event/WriteMetadataEvent.java | 0 .../org/apache/hudi/sink/meta/CkpMessage.java | 0 .../apache/hudi/sink/meta/CkpMetadata.java | 3 +- .../partitioner/BucketAssignFunction.java | 0 .../hudi/sink/partitioner/BucketAssigner.java | 0 .../sink/partitioner/BucketAssigners.java | 0 .../partitioner/BucketIndexPartitioner.java | 3 +- .../profile/DeltaWriteProfile.java | 0 .../profile/EmptyWriteProfile.java | 0 .../partitioner/profile/WriteProfile.java | 0 .../partitioner/profile/WriteProfiles.java | 11 +- .../sink/transform/ChainedTransformer.java | 0 .../transform/RowDataToHoodieFunction.java | 0 .../RowDataToHoodieFunctionWithRateLimit.java | 6 +- .../transform/RowDataToHoodieFunctions.java | 0 .../hudi/sink/transform/Transformer.java | 0 .../hudi/sink/utils/HiveSyncContext.java | 0 .../hudi/sink/utils/NonThrownExecutor.java | 0 .../hudi/sink/utils/PayloadCreation.java | 0 .../org/apache/hudi/sink/utils/Pipelines.java | 50 +-- .../org/apache/hudi/sink/utils/TimeWait.java | 0 .../org/apache/hudi/source/FileIndex.java | 0 .../hudi/source/IncrementalInputSplits.java | 0 .../source/StreamReadMonitoringFunction.java | 0 .../hudi/source/StreamReadOperator.java | 38 +- .../hudi/streamer/FlinkStreamerConfig.java | 0 .../hudi/streamer/HoodieFlinkStreamer.java | 16 +- .../apache/hudi/table/HoodieTableFactory.java | 0 .../apache/hudi/table/HoodieTableSink.java | 0 .../apache/hudi/table/HoodieTableSource.java | 0 .../hudi/table/catalog/CatalogOptions.java | 0 .../hudi/table/catalog/HoodieCatalog.java | 0 .../table/catalog/HoodieCatalogFactory.java | 0 .../table/catalog/TableOptionProperties.java | 0 .../hudi/table/format/FilePathUtils.java | 0 .../apache/hudi/table/format/FormatUtils.java | 2 +- .../format/cow/CopyOnWriteInputFormat.java | 0 .../format/cow/ParquetSplitReaderUtil.java | 2 +- .../format/cow/vector/HeapArrayVector.java | 0 .../cow/vector/HeapMapColumnVector.java | 0 .../cow/vector/HeapRowColumnVector.java | 0 .../cow/vector/ParquetDecimalVector.java | 0 .../vector/reader/AbstractColumnReader.java | 0 .../cow/vector/reader/ArrayColumnReader.java | 0 .../reader/BaseVectorizedColumnReader.java | 0 .../reader/FixedLenBytesColumnReader.java | 0 .../reader/Int64TimestampColumnReader.java | 0 .../cow/vector/reader/MapColumnReader.java | 0 .../reader/ParquetColumnarRowSplitReader.java | 0 .../reader/ParquetDataColumnReader.java | 0 .../ParquetDataColumnReaderFactory.java | 0 .../cow/vector/reader/RowColumnReader.java | 0 .../cow/vector/reader/RunLengthDecoder.java | 0 .../format/mor/MergeOnReadInputFormat.java | 2 +- .../format/mor/MergeOnReadInputSplit.java | 0 .../format/mor/MergeOnReadTableState.java | 0 .../apache/hudi/util/AvroSchemaConverter.java | 4 +- .../hudi/util/AvroToRowDataConverters.java | 0 .../org/apache/hudi/util/ChangelogModes.java | 0 .../org/apache/hudi/util/CompactionUtil.java | 6 +- .../org/apache/hudi/util/DataTypeUtils.java | 0 .../hudi/util/FlinkStateBackendConverter.java | 0 .../org/apache/hudi/util/FlinkTables.java | 0 .../org/apache/hudi/util/InputFormats.java | 0 .../apache/hudi/util/RowDataProjection.java | 0 .../hudi/util/RowDataToAvroConverters.java | 0 .../org/apache/hudi/util/StreamerUtil.java | 1 + .../hudi/util/StringToRowDataConverter.java | 2 +- .../hudi/util/ViewStorageProperties.java | 0 .../org.apache.flink.table.factories.Factory | 0 .../hudi/sink/ITTestDataStreamWrite.java | 2 +- .../TestStreamWriteOperatorCoordinator.java | 17 +- .../hudi/sink/TestWriteCopyOnWrite.java | 2 +- .../hudi/sink/TestWriteMergeOnRead.java | 0 .../sink/TestWriteMergeOnReadWithCompact.java | 0 .../hudi/sink/bulk/TestRowDataKeyGen.java | 0 .../compact/ITTestHoodieFlinkCompactor.java | 0 .../hudi/sink/meta/TestCkpMetadata.java | 0 .../sink/partitioner/TestBucketAssigner.java | 0 .../hudi/sink/utils/CollectorOutput.java | 39 +- .../sink/utils/CompactFunctionWrapper.java | 41 +- .../sink/utils/InsertFunctionWrapper.java | 0 .../sink/utils/MockCoordinatorExecutor.java | 0 .../apache/hudi/sink/utils/MockMapState.java | 0 .../sink/utils/MockOperatorStateStore.java | 0 .../utils/MockStateInitializationContext.java | 12 +- .../utils/MockStreamingRuntimeContext.java | 11 +- .../hudi/sink/utils/MockValueState.java | 0 .../utils/StreamWriteFunctionWrapper.java | 77 ++-- .../hudi/sink/utils/TestFunctionWrapper.java | 0 .../apache/hudi/sink/utils/TestWriteBase.java | 0 .../org/apache/hudi/source/TestFileIndex.java | 0 .../TestStreamReadMonitoringFunction.java | 0 .../hudi/source/TestStreamReadOperator.java | 0 .../hudi/table/ITTestHoodieDataSource.java | 22 +- .../hudi/table/TestHoodieTableFactory.java | 0 .../hudi/table/TestHoodieTableSource.java | 0 .../hudi/table/catalog/TestHoodieCatalog.java | 3 +- .../hudi/table/format/TestInputFormat.java | 0 .../org/apache/hudi/utils/SchemaBuilder.java | 0 .../apache/hudi/utils/TestCompactionUtil.java | 0 .../apache/hudi/utils/TestConfigurations.java | 22 +- .../java/org/apache/hudi/utils/TestData.java | 0 .../apache/hudi/utils/TestHoodieRowData.java | 22 +- .../java/org/apache/hudi/utils/TestSQL.java | 0 .../apache/hudi/utils/TestStreamerUtil.java | 0 .../utils/TestStringToRowDataConverter.java | 0 .../java/org/apache/hudi/utils/TestUtils.java | 0 .../hudi/utils/TestViewStorageProperties.java | 0 .../factory/CollectSinkTableFactory.java | 0 .../factory/ContinuousFileSourceFactory.java | 0 .../utils/source/ContinuousFileSource.java | 0 .../org.apache.flink.table.factories.Factory | 0 .../src/test/resources/debezium_json.data | 0 .../resources/log4j-surefire-quiet.properties | 1 - .../test/resources/log4j-surefire.properties | 1 - .../src/test/resources/test_read_schema.avsc | 0 .../src/test/resources/test_source.data | 0 .../src/test/resources/test_source_2.data | 0 .../src/test/resources/test_source_3.data | 0 .../src/test/resources/test_source_4.data | 0 .../src/test/resources/test_source_5.data | 0 .../hudi-flink1.13.x/pom.xml | 90 +++++ .../flink/table/data/ColumnarArrayData.java | 262 +++++++++++++ .../flink/table/data/ColumnarMapData.java | 71 ++++ .../flink/table/data/ColumnarRowData.java | 223 +++++++++++ .../table/data/vector/MapColumnVector.java | 28 ++ .../table/data/vector/RowColumnVector.java | 28 ++ .../data/vector/VectorizedColumnBatch.java | 136 +++++++ .../AbstractStreamOperatorAdapter.java | 35 ++ .../AbstractStreamOperatorFactoryAdapter.java | 50 +++ .../hudi/adapter/MailboxExecutorAdapter.java | 37 ++ .../hudi/adapter/RateLimiterAdapter.java | 40 ++ .../java/org/apache/hudi/adapter/Utils.java | 48 +++ .../apache/hudi/adapter/OutputAdapter.java | 27 ++ .../StateInitializationContextAdapter.java | 26 ++ .../StreamingRuntimeContextAdapter.java | 43 +++ .../apache/hudi/adapter/TestTableEnvs.java | 34 ++ .../hudi-flink1.14.x/pom.xml | 102 +++++ .../AbstractStreamOperatorAdapter.java | 27 ++ .../AbstractStreamOperatorFactoryAdapter.java | 33 ++ .../hudi/adapter/MailboxExecutorAdapter.java | 37 ++ .../hudi/adapter/RateLimiterAdapter.java | 40 ++ .../java/org/apache/hudi/adapter/Utils.java | 48 +++ .../apache/hudi/adapter/OutputAdapter.java | 32 ++ .../StateInitializationContextAdapter.java | 32 ++ .../StreamingRuntimeContextAdapter.java | 43 +++ .../apache/hudi/adapter/TestTableEnvs.java | 52 +++ hudi-flink-datasource/pom.xml | 41 ++ hudi-flink/pom.xml | 349 ----------------- hudi-kafka-connect/pom.xml | 2 +- packaging/hudi-flink-bundle/pom.xml | 12 +- pom.xml | 42 +- 193 files changed, 2263 insertions(+), 628 deletions(-) create mode 100644 hudi-flink-datasource/hudi-flink/pom.xml rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/configuration/FlinkOptions.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/configuration/OptionsResolver.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/schema/SchemaProvider.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java (97%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/CleanFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/append/AppendWriteOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java (98%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAccumulator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAggFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bootstrap/batch/BatchBootstrapOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/common/AbstractWriteOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/common/WriteOperatorFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactionCommitEvent.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactionPlanEvent.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/event/CommitAckEvent.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/event/WriteMetadataEvent.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/meta/CkpMessage.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigners.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/profile/EmptyWriteProfile.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java (98%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/transform/ChainedTransformer.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java (89%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctions.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/transform/Transformer.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/utils/NonThrownExecutor.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/utils/Pipelines.java (90%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/sink/utils/TimeWait.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/source/FileIndex.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/source/StreamReadOperator.java (89%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java (94%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/HoodieTableFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/HoodieTableSink.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/HoodieTableSource.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/FilePathUtils.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/FormatUtils.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/ChangelogModes.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/CompactionUtil.java (97%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/DataTypeUtils.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/FlinkTables.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/InputFormats.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/RowDataProjection.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/StreamerUtil.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/java/org/apache/hudi/util/ViewStorageProperties.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java (62%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java (79%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/InsertFunctionWrapper.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockCoordinatorExecutor.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockMapState.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockOperatorStateStore.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java (89%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java (93%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/MockValueState.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java (88%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/TestFunctionWrapper.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/source/TestFileIndex.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java (97%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java (98%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/table/format/TestInputFormat.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/SchemaBuilder.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestConfigurations.java (93%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestData.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java (90%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestSQL.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestStringToRowDataConverter.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestUtils.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/TestViewStorageProperties.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/factory/CollectSinkTableFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/factory/ContinuousFileSourceFactory.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/debezium_json.data (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/log4j-surefire-quiet.properties (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/log4j-surefire.properties (99%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_read_schema.avsc (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_source.data (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_source_2.data (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_source_3.data (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_source_4.data (100%) rename {hudi-flink => hudi-flink-datasource/hudi-flink}/src/test/resources/test_source_5.data (100%) create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/pom.xml create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarMapData.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarRowData.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/pom.xml create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java create mode 100644 hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java create mode 100644 hudi-flink-datasource/pom.xml delete mode 100644 hudi-flink/pom.xml diff --git a/azure-pipelines.yml b/azure-pipelines.yml index c2d5b510de2a3..397ff9800c4b3 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -57,7 +57,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Punit-tests -pl hudi-common,hudi-flink,hudi-client/hudi-spark-client + options: -Punit-tests -pl hudi-common,hudi-flink-datasource/hudi-flink,hudi-client/hudi-spark-client publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -66,7 +66,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Pfunctional-tests -pl hudi-common,hudi-flink + options: -Pfunctional-tests -pl hudi-common,hudi-flink-datasource/hudi-flink publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -165,7 +165,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Punit-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync + options: -Punit-tests -pl !hudi-common,!hudi-flink-datasource/hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' @@ -174,7 +174,7 @@ stages: inputs: mavenPomFile: 'pom.xml' goals: 'test' - options: -Pfunctional-tests -pl !hudi-common,!hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync + options: -Pfunctional-tests -pl !hudi-common,!hudi-flink-datasource/hudi-flink,!hudi-client/hudi-spark-client,!hudi-client/hudi-client-common,!hudi-client/hudi-flink-client,!hudi-client/hudi-java-client,!hudi-cli,!hudi-utilities,!hudi-sync/hudi-hive-sync publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index b6f1f3d372d28..ffe82a0c96b59 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -60,7 +60,7 @@
    org.apache.flink - flink-table-runtime_${scala.binary.version} + ${flink.table.runtime.artifactId} ${flink.version} provided @@ -159,7 +159,7 @@ org.apache.flink - flink-runtime + ${flink.runtime.artifactId} ${flink.version} test tests diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml new file mode 100644 index 0000000000000..bd05e7c10a4d7 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -0,0 +1,364 @@ + + + + + + + hudi-flink-datasource + org.apache.hudi + 0.11.0-SNAPSHOT + + 4.0.0 + + hudi-flink + 0.11.0-SNAPSHOT + jar + + + ${project.parent.parent.basedir} + 1.11.1 + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-jar-plugin + 3.1.2 + + + + test-jar + + + + + + org.apache.rat + apache-rat-plugin + + + + + + src/main/resources + + + src/test/resources + + + + + + + + org.apache.hudi + hudi-common + ${project.version} + + + org.apache.hudi + hudi-client-common + ${project.version} + + + org.apache.hudi + hudi-flink-client + ${project.version} + + + org.apache.hudi + hudi-hadoop-mr + ${project.version} + + + org.apache.hudi + hudi-hive-sync + ${project.version} + + + org.apache.hudi + hudi-sync-common + ${project.version} + + + org.apache.hudi + ${hudi.flink.module} + ${project.version} + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + compile + + + org.apache.flink + flink-clients_${scala.binary.version} + compile + + + com.esotericsoftware.kryo + kryo + + + com.esotericsoftware.minlog + minlog + + + + + org.apache.flink + flink-connector-kafka_${scala.binary.version} + compile + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.flink + flink-hadoop-compatibility_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-parquet_${scala.binary.version} + ${flink.version} + provided + + + org.apache.flink + flink-json + ${flink.version} + provided + + + org.apache.flink + flink-table-common + ${flink.version} + provided + + + org.apache.flink + ${flink.table.runtime.artifactId} + ${flink.version} + provided + + + org.apache.flink + ${flink.table.planner.artifactId} + ${flink.version} + provided + + + org.apache.flink + flink-statebackend-rocksdb_${scala.binary.version} + ${flink.version} + provided + + + + org.apache.parquet + parquet-hadoop + ${parquet.version} + + + org.xerial.snappy + snappy-java + + + + + + + org.apache.parquet + parquet-avro + ${parquet.version} + test + + + + + org.apache.avro + avro + + 1.10.0 + compile + + + + + org.apache.hadoop + hadoop-mapreduce-client-core + compile + + + org.slf4j + slf4j-log4j12 + + + + + + com.beust + jcommander + compile + + + com.twitter + bijection-avro_${scala.binary.version} + 0.9.7 + + + joda-time + joda-time + 2.5 + + + + ${hive.groupid} + hive-exec + ${hive.version} + ${hive.exec.classifier} + + + javax.mail + mail + + + org.eclipse.jetty.aggregate + * + + + + + + + + + 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.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.flink.module} + ${project.version} + tests + test-jar + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${flink.version} + test + + + org.apache.flink + ${flink.runtime.artifactId} + ${flink.version} + test + test-jar + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + test + test-jar + + + org.apache.flink + ${flink.table.runtime.artifactId} + ${flink.version} + test + test-jar + + + org.apache.flink + flink-json + ${flink.version} + test + test-jar + + + org.apache.flink + flink-csv + ${flink.version} + test + + + diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index 90ed73a3deb83..cd5c2a70e1d48 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -323,8 +323,8 @@ private FlinkOptions() { .stringType() .defaultValue("") .withDescription("Index key field. Value to be used as hashing to find the bucket ID. Should be a subset of or equal to the recordKey fields.\n" - + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using " - + "the dot notation eg: `a.b.c`"); + + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using " + + "the dot notation eg: `a.b.c`"); public static final ConfigOption BUCKET_INDEX_NUM_BUCKETS = ConfigOptions .key(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key()) diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/FilebasedSchemaProvider.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaProvider.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/schema/SchemaRegistryProvider.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java similarity index 97% rename from hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java index 128358096cde6..057c794331d66 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java @@ -18,10 +18,6 @@ package org.apache.hudi.sink; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.streaming.api.functions.ProcessFunction; -import org.apache.flink.util.Collector; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -31,6 +27,11 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.index.bucket.BucketIdentifier; import org.apache.hudi.table.HoodieFlinkTable; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -131,7 +132,7 @@ private void bootstrapIndex() throws IOException { int partitionOfBucket = BucketIdentifier.mod(i, parallelism); if (partitionOfBucket == taskID) { LOG.info(String.format("Bootstrapping index. Adding bucket %s , " - + "Current parallelism: %s , Max parallelism: %s , Current task id: %s", + + "Current parallelism: %s , Max parallelism: %s , Current task id: %s", i, parallelism, maxParallelism, taskID)); bucketToLoad.add(i); } @@ -155,7 +156,7 @@ private void bootstrapIndex() throws IOException { LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID)); if (bucketToFileIDMap.containsKey(partitionBucketId)) { throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found " - + "during the BucketStreamWriteFunction index bootstrap.", fileID, partitionBucketId)); + + "during the BucketStreamWriteFunction index bootstrap.", fileID, partitionBucketId)); } else { LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId)); bucketToFileIDMap.put(partitionBucketId, fileID); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java index 209fe59e4b8ca..cf740cc2ccc59 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteOperator.java @@ -18,10 +18,11 @@ package org.apache.hudi.sink; -import org.apache.flink.configuration.Configuration; import org.apache.hudi.sink.common.AbstractWriteOperator; import org.apache.hudi.sink.common.WriteOperatorFactory; +import org.apache.flink.configuration.Configuration; + /** * Operator for {@link BucketStreamWriteFunction}. * diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteOperator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteOperator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java similarity index 98% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index cdb378f88885d..1fc8d393be6a9 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -256,9 +256,9 @@ public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice f } protected boolean shouldLoadFile(String fileId, - int maxParallelism, - int parallelism, - int taskID) { + int maxParallelism, + int parallelism, + int taskID) { return KeyGroupRangeAssignment.assignKeyToParallelOperator( fileId, maxParallelism, parallelism) == taskID; } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/IndexRecord.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAccumulator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAccumulator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAccumulator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAccumulator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAggFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAggFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAggFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/aggregate/BootstrapAggFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/batch/BatchBootstrapOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/batch/BatchBootstrapOperator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/batch/BatchBootstrapOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/batch/BatchBootstrapOperator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteOperator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteOperator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriterHelper.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperator.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperator.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/sort/SortOperatorGen.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java index 8e776006f3b9b..9e131ff91e1ea 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractWriteFunction.java @@ -41,6 +41,7 @@ public abstract class AbstractWriteFunction extends ProcessFunction uniqueIdToFileStatus = new HashMap<>(); metadataList.forEach(metadata -> - uniqueIdToFileStatus.putAll(getFilesToReadOfInstant(basePath, metadata, fs, tableType))); + uniqueIdToFileStatus.putAll(getFilesToReadOfInstant(basePath, metadata, fs, tableType))); return uniqueIdToFileStatus.values().toArray(new FileStatus[0]); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/ChainedTransformer.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/ChainedTransformer.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/transform/ChainedTransformer.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/ChainedTransformer.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java similarity index 89% rename from hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java index 3d42ad87d908e..fc9c2177e7c0b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/transform/RowDataToHoodieFunctionWithRateLimit.java @@ -18,11 +18,11 @@ package org.apache.hudi.sink.transform; +import org.apache.hudi.adapter.RateLimiterAdapter; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.configuration.FlinkOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -39,7 +39,7 @@ public class RowDataToHoodieFunctionWithRateLimit bootstrap( * The bootstrap operator loads the existing data index (primary key to file id mapping), * then send the indexing data set to subsequent operator(usually the bucket assign operator). * - * @param conf The configuration - * @param rowType The row type + * @param conf The configuration + * @param rowType The row type * @param defaultParallelism The default parallelism - * @param dataStream The data stream - * @param bounded Whether the source is bounded - * @param overwrite Whether it is insert overwrite + * @param dataStream The data stream + * @param bounded Whether the source is bounded + * @param overwrite Whether it is insert overwrite */ public static DataStream bootstrap( Configuration conf, @@ -268,9 +268,9 @@ public static DataStream rowDataToHoodieRecord(Configuration conf, *

    The bucket assigner assigns the inputs to suitable file groups, the write task caches * and flushes the data set to disk. * - * @param conf The configuration + * @param conf The configuration * @param defaultParallelism The default parallelism - * @param dataStream The input data stream + * @param dataStream The input data stream * @return the stream write data stream pipeline */ public static DataStream hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream dataStream) { @@ -280,25 +280,25 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD); BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields); return dataStream.partitionCustom(partitioner, HoodieRecord::getKey) - .transform("bucket_write", TypeInformation.of(Object.class), operatorFactory) - .uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME)) - .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); + .transform("bucket_write", TypeInformation.of(Object.class), operatorFactory) + .uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME)) + .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } else { WriteOperatorFactory operatorFactory = StreamWriteOperator.getFactory(conf); return dataStream - // Key-by record key, to avoid multiple subtasks write to a bucket at the same time - .keyBy(HoodieRecord::getRecordKey) - .transform( - "bucket_assigner", - TypeInformation.of(HoodieRecord.class), - new KeyedProcessOperator<>(new BucketAssignFunction<>(conf))) - .uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME)) - .setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism)) - // shuffle by fileId(bucket id) - .keyBy(record -> record.getCurrentLocation().getFileId()) - .transform("stream_write", TypeInformation.of(Object.class), operatorFactory) - .uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) - .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); + // Key-by record key, to avoid multiple subtasks write to a bucket at the same time + .keyBy(HoodieRecord::getRecordKey) + .transform( + "bucket_assigner", + TypeInformation.of(HoodieRecord.class), + new KeyedProcessOperator<>(new BucketAssignFunction<>(conf))) + .uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME)) + .setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism)) + // shuffle by fileId(bucket id) + .keyBy(record -> record.getCurrentLocation().getFileId()) + .transform("stream_write", TypeInformation.of(Object.class), operatorFactory) + .uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME)) + .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); } } @@ -324,8 +324,8 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau */ public static DataStreamSink compact(Configuration conf, DataStream dataStream) { return dataStream.transform("compact_plan_generate", - TypeInformation.of(CompactionPlanEvent.class), - new CompactionPlanOperator(conf)) + TypeInformation.of(CompactionPlanEvent.class), + new CompactionPlanOperator(conf)) .setParallelism(1) // plan generate must be singleton .rebalance() .transform("compact_task", diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/TimeWait.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/TimeWait.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/sink/utils/TimeWait.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/TimeWait.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java similarity index 89% rename from hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java index c3f43422f1d1e..bf6dc98f4de06 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadOperator.java @@ -18,24 +18,23 @@ package org.apache.hudi.source; +import org.apache.hudi.adapter.AbstractStreamOperatorAdapter; +import org.apache.hudi.adapter.AbstractStreamOperatorFactoryAdapter; +import org.apache.hudi.adapter.MailboxExecutorAdapter; +import org.apache.hudi.adapter.Utils; import org.apache.hudi.table.format.mor.MergeOnReadInputFormat; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; -import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.runtime.state.JavaSerializer; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.StreamSourceContexts; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; @@ -55,11 +54,11 @@ * this operator can have multiple parallelism. * *

    As soon as an input split {@link MergeOnReadInputSplit} is received, it is put into a queue, - * the {@link MailboxExecutor} read the actual data of the split. + * the {@code MailboxExecutor} read the actual data of the split. * This architecture allows the separation of split reading from processing the checkpoint barriers, * thus removing any potential back-pressure. */ -public class StreamReadOperator extends AbstractStreamOperator +public class StreamReadOperator extends AbstractStreamOperatorAdapter implements OneInputStreamOperator { private static final Logger LOG = LoggerFactory.getLogger(StreamReadOperator.class); @@ -69,7 +68,7 @@ public class StreamReadOperator extends AbstractStreamOperator // It's the same thread that runs this operator and checkpoint actions. Use this executor to schedule only // splits for subsequent reading, so that a new checkpoint could be triggered without blocking a long time // for exhausting all scheduled split reading tasks. - private final MailboxExecutor executor; + private final MailboxExecutorAdapter executor; private MergeOnReadInputFormat format; @@ -86,7 +85,7 @@ public class StreamReadOperator extends AbstractStreamOperator private transient volatile SplitState currentSplitState; private StreamReadOperator(MergeOnReadInputFormat format, ProcessingTimeService timeService, - MailboxExecutor mailboxExecutor) { + MailboxExecutorAdapter mailboxExecutor) { this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); this.processingTimeService = timeService; this.executor = Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); @@ -114,14 +113,12 @@ public void initializeState(StateInitializationContext context) throws Exception } } - this.sourceContext = StreamSourceContexts.getSourceContext( + this.sourceContext = Utils.getSourceContext( getOperatorConfig().getTimeCharacteristic(), getProcessingTimeService(), - new Object(), // no actual locking needed + getContainingTask(), output, - getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), - -1, - true); + getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval()); // Enqueue to process the recovered input splits. enqueueProcessSplits(); @@ -236,26 +233,19 @@ private enum SplitState { IDLE, RUNNING } - private static class OperatorFactory extends AbstractStreamOperatorFactory - implements YieldingOperatorFactory, OneInputStreamOperatorFactory { + private static class OperatorFactory extends AbstractStreamOperatorFactoryAdapter + implements OneInputStreamOperatorFactory { private final MergeOnReadInputFormat format; - private transient MailboxExecutor mailboxExecutor; - private OperatorFactory(MergeOnReadInputFormat format) { this.format = format; } - @Override - public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { - this.mailboxExecutor = mailboxExecutor; - } - @SuppressWarnings("unchecked") @Override public > O createStreamOperator(StreamOperatorParameters parameters) { - StreamReadOperator operator = new StreamReadOperator(format, processingTimeService, mailboxExecutor); + StreamReadOperator operator = new StreamReadOperator(format, processingTimeService, getMailboxExecutorAdapter()); operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); return (O) operator; } diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java similarity index 94% rename from hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java index 192de91d238a3..a12ec23dcb036 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java @@ -79,14 +79,14 @@ public static void main(String[] args) throws Exception { conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); DataStream dataStream = env.addSource(new FlinkKafkaConsumer<>( - cfg.kafkaTopic, - new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601 - ), kafkaProps)) + cfg.kafkaTopic, + new JsonRowDataDeserializationSchema( + rowType, + InternalTypeInfo.of(rowType), + false, + true, + TimestampFormat.ISO_8601 + ), kafkaProps)) .name("kafka_source") .uid("uid_kafka_source"); diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/CatalogOptions.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index 666dc3a73fb90..fce9b75f764ea 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -33,6 +33,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; +import org.apache.hudi.util.StreamerUtil; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -42,7 +43,6 @@ import org.apache.flink.types.RowKind; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hudi.util.StreamerUtil; import java.util.ArrayList; import java.util.Arrays; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index e112bcf24e50f..c636b36100fea 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -334,7 +334,7 @@ private static ColumnReader createColumnReader( case TIMESTAMP_WITH_LOCAL_TIME_ZONE: switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { case INT64: - return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType)fieldType).getPrecision()); + return new Int64TimestampColumnReader(utcTimestamp, descriptor, pageReader, ((TimestampType) fieldType).getPrecision()); case INT96: return new TimestampColumnReader(utcTimestamp, descriptor, pageReader); default: diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/BaseVectorizedColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/MapColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetDataColumnReaderFactory.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RowColumnReader.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 8283b5c3cd5b1..202b14404aa35 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -31,8 +31,8 @@ import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.format.FilePathUtils; import org.apache.hudi.table.format.FormatUtils; -import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; import org.apache.hudi.table.format.cow.ParquetSplitReaderUtil; +import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; import org.apache.hudi.util.AvroToRowDataConverters; import org.apache.hudi.util.RowDataProjection; import org.apache.hudi.util.RowDataToAvroConverters; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java index 3328233e63e83..6325c2bcceb4e 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java @@ -76,8 +76,8 @@ public static DataType convertToDataType(Schema schema) { return DataTypes.ARRAY(convertToDataType(schema.getElementType())).notNull(); case MAP: return DataTypes.MAP( - DataTypes.STRING().notNull(), - convertToDataType(schema.getValueType())) + DataTypes.STRING().notNull(), + convertToDataType(schema.getValueType())) .notNull(); case UNION: final Schema actualSchema; diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/ChangelogModes.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ChangelogModes.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/ChangelogModes.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ChangelogModes.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java similarity index 97% rename from hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java index 74629f9b0942f..3d386cf8cc175 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java @@ -51,10 +51,10 @@ public class CompactionUtil { /** * Schedules a new compaction instant. * - * @param metaClient The metadata client - * @param writeClient The write client + * @param metaClient The metadata client + * @param writeClient The write client * @param deltaTimeCompaction Whether the compaction is trigger by elapsed delta time - * @param committed Whether the last instant was committed successfully + * @param committed Whether the last instant was committed successfully */ public static void scheduleCompaction( HoodieTableMetaClient metaClient, diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/DataTypeUtils.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkStateBackendConverter.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkTables.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/InputFormats.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/InputFormats.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/InputFormats.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/InputFormats.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataProjection.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 55d403dc4db8d..3138d5d986167 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -503,6 +503,7 @@ public static String getLastCompletedInstant(HoodieTableMetaClient metaClient) { /** * Returns whether there are successful commits on the timeline. + * * @param metaClient The meta client * @return true if there is any successful commit */ diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java index 47aed1ed31e77..216fa3f0f336f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StringToRowDataConverter.java @@ -18,7 +18,6 @@ package org.apache.hudi.util; -import org.apache.flink.table.types.logical.TimestampType; import org.apache.hudi.common.util.ValidationUtils; import org.apache.flink.annotation.Internal; @@ -27,6 +26,7 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimestampType; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java similarity index 100% rename from hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java rename to hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java diff --git a/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to hudi-flink-datasource/hudi-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java similarity index 99% rename from hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java index 4864696da144c..bbf9009fd5b75 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java @@ -178,7 +178,7 @@ public void testMergeOnReadWriteWithCompaction(String indexType) throws Exceptio DataStream pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream); Pipelines.clean(conf, pipeline); Pipelines.compact(conf, pipeline); - JobClient client = execEnv.executeAsync(execEnv.getStreamGraph()); + JobClient client = execEnv.executeAsync("mor-write-with-compact"); if (client.getJobStatus().get() != JobStatus.FAILED) { try { TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java similarity index 99% rename from hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index 6266c30523092..814a8f19e1596 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -18,14 +18,6 @@ package org.apache.hudi.sink; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.util.FileUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieWriteStat; @@ -38,6 +30,15 @@ import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestUtils; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.FileUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java similarity index 99% rename from hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java index 08035aff5a167..4771a7a3455b0 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java @@ -119,7 +119,7 @@ public void testInsert() throws Exception { // open the function and ingest data preparePipeline() .consume(TestData.DATA_SET_INSERT) - .assertEmptyDataFiles() + .assertEmptyDataFiles() .checkpoint(1) .assertNextEvent() .checkpointComplete(1) diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java similarity index 62% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java index c386e6287b8cd..b18cfac51b44f 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CollectorOutput.java @@ -18,59 +18,45 @@ package org.apache.hudi.sink.utils; +import org.apache.hudi.adapter.OutputAdapter; + import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; -import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; -import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.OutputTag; -import java.io.IOException; +import java.util.ArrayList; import java.util.List; /** * Collecting {@link Output} for {@link StreamRecord}. */ -public class CollectorOutput implements Output> { +public class CollectorOutput implements OutputAdapter> { - private final List list; + private final List records; - public CollectorOutput(List list) { - this.list = list; + public CollectorOutput() { + this.records = new ArrayList<>(); } - public List getList() { - return list; + public List getRecords() { + return this.records; } @Override public void emitWatermark(Watermark mark) { - list.add(mark); - } - - @Override - public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { - + // no operation } @Override public void emitLatencyMarker(LatencyMarker latencyMarker) { - list.add(latencyMarker); + // no operation } @Override public void collect(StreamRecord record) { - try { - ClassLoader cl = record.getClass().getClassLoader(); - T copied = - InstantiationUtil.deserializeObject( - InstantiationUtil.serializeObject(record.getValue()), cl); - list.add(record.copy(copied)); - } catch (IOException | ClassNotFoundException ex) { - throw new RuntimeException("Unable to deserialize record: " + record, ex); - } + records.add(record.getValue()); } @Override @@ -80,5 +66,6 @@ public void collect(OutputTag outputTag, StreamRecord record) { @Override public void close() { + this.records.clear(); } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java similarity index 79% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java index e703515de3b7f..1dba81ce2b7b6 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/CompactFunctionWrapper.java @@ -33,14 +33,8 @@ import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.Collector; -import org.apache.flink.util.OutputTag; import java.util.ArrayList; import java.util.List; @@ -95,44 +89,13 @@ public void openFunction() throws Exception { } public void compact(long checkpointID) throws Exception { - List events = new ArrayList<>(); // collect the CompactEvents. - Output> output = new Output>() { - @Override - public void emitWatermark(Watermark watermark) { - - } - - @Override - public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { - - } - - @Override - public void collect(OutputTag outputTag, StreamRecord streamRecord) { - - } - - @Override - public void emitLatencyMarker(LatencyMarker latencyMarker) { - - } - - @Override - public void collect(StreamRecord record) { - events.add(record.getValue()); - } - - @Override - public void close() { - - } - }; + CollectorOutput output = new CollectorOutput<>(); compactionPlanOperator.setOutput(output); compactionPlanOperator.notifyCheckpointComplete(checkpointID); // collect the CompactCommitEvents List compactCommitEvents = new ArrayList<>(); - for (CompactionPlanEvent event : events) { + for (CompactionPlanEvent event : output.getRecords()) { compactFunction.processElement(event, null, new Collector() { @Override public void collect(CompactionCommitEvent event) { diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/InsertFunctionWrapper.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/InsertFunctionWrapper.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/InsertFunctionWrapper.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/InsertFunctionWrapper.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockCoordinatorExecutor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockCoordinatorExecutor.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockCoordinatorExecutor.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockCoordinatorExecutor.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockMapState.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockMapState.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockMapState.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockMapState.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockOperatorStateStore.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockOperatorStateStore.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockOperatorStateStore.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockOperatorStateStore.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java similarity index 89% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java index c582e9553b30e..945d1bbbe75f5 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStateInitializationContext.java @@ -17,18 +17,17 @@ package org.apache.hudi.sink.utils; +import org.apache.hudi.adapter.StateInitializationContextAdapter; + import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; -import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StatePartitionStreamProvider; -import java.util.OptionalLong; - /** * A {@link FunctionInitializationContext} for testing purpose. */ -public class MockStateInitializationContext implements StateInitializationContext { +public class MockStateInitializationContext implements StateInitializationContextAdapter { private final MockOperatorStateStore operatorStateStore; @@ -41,11 +40,6 @@ public boolean isRestored() { return false; } - @Override - public OptionalLong getRestoredCheckpointId() { - return OptionalLong.empty(); - } - @Override public MockOperatorStateStore getOperatorStateStore() { return operatorStateStore; diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java similarity index 93% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java index 8a66f1dce011a..7c5b79700e434 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockStreamingRuntimeContext.java @@ -17,10 +17,10 @@ package org.apache.hudi.sink.utils; +import org.apache.hudi.adapter.StreamingRuntimeContextAdapter; + import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.KeyedStateStore; -import org.apache.flink.metrics.groups.OperatorMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.MockEnvironment; @@ -37,7 +37,7 @@ * *

    NOTE: Adapted from Apache Flink, the MockStreamOperator is modified to support MapState. */ -public class MockStreamingRuntimeContext extends StreamingRuntimeContext { +public class MockStreamingRuntimeContext extends StreamingRuntimeContextAdapter { private final boolean isCheckpointingEnabled; @@ -68,11 +68,6 @@ public MockStreamingRuntimeContext( this.subtaskIndex = subtaskIndex; } - @Override - public OperatorMetricGroup getMetricGroup() { - return UnregisteredMetricsGroup.createOperatorMetricGroup(); - } - @Override public boolean isCheckpointingEnabled() { return isCheckpointingEnabled; diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockValueState.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockValueState.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockValueState.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/MockValueState.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java similarity index 88% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java index 2c8bac0576682..2bb0f69d18ca9 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java @@ -42,18 +42,14 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.operators.collect.utils.MockFunctionSnapshotContext; import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway; -import org.apache.flink.streaming.runtime.streamrecord.StreamElement; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.MockStreamTask; import org.apache.flink.streaming.util.MockStreamTaskBuilder; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Collector; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -98,8 +94,6 @@ public class StreamWriteFunctionWrapper implements TestFunctionWrapper { private CompactFunctionWrapper compactFunctionWrapper; - private final Output>> output; - private final MockStreamTask streamTask; private final StreamConfig streamConfig; @@ -128,7 +122,6 @@ public StreamWriteFunctionWrapper(String tablePath, Configuration conf) throws E this.stateInitializationContext = new MockStateInitializationContext(); this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf); this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf); - this.output = new CollectorOutput<>(new ArrayList<>()); this.streamConfig = new StreamConfig(conf); streamConfig.setOperatorID(new OperatorID()); this.streamTask = new MockStreamTaskBuilder(environment) @@ -144,16 +137,23 @@ public void openFunction() throws Exception { toHoodieFunction.setRuntimeContext(runtimeContext); toHoodieFunction.open(conf); + bucketAssignerFunction = new BucketAssignFunction<>(conf); + bucketAssignerFunction.setRuntimeContext(runtimeContext); + bucketAssignerFunction.open(conf); + bucketAssignerFunction.initializeState(this.stateInitializationContext); + if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { bootstrapOperator = new BootstrapOperator<>(conf); + CollectorOutput> output = new CollectorOutput<>(); bootstrapOperator.setup(streamTask, streamConfig, output); bootstrapOperator.initializeState(this.stateInitializationContext); - } - bucketAssignerFunction = new BucketAssignFunction<>(conf); - bucketAssignerFunction.setRuntimeContext(runtimeContext); - bucketAssignerFunction.open(conf); - bucketAssignerFunction.initializeState(this.stateInitializationContext); + Collector> collector = ScalaCollector.getInstance(); + for (HoodieRecord bootstrapRecord : output.getRecords()) { + bucketAssignerFunction.processElement(bootstrapRecord, null, collector); + bucketAssignFunctionContext.setCurrentKey(bootstrapRecord.getRecordKey()); + } + } setupWriteFunction(); @@ -162,39 +162,12 @@ public void openFunction() throws Exception { } } - @SuppressWarnings("unchecked") public void invoke(I record) throws Exception { HoodieRecord hoodieRecord = toHoodieFunction.map((RowData) record); - HoodieRecord[] hoodieRecords = new HoodieRecord[1]; - Collector> collector = new Collector>() { - @Override - public void collect(HoodieRecord record) { - hoodieRecords[0] = record; - } - - @Override - public void close() { - - } - }; - - if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { - List list = ((CollectorOutput) output).getList(); - for (StreamElement streamElement : list) { - if (streamElement.isRecord()) { - HoodieRecord bootstrapRecord = (HoodieRecord) streamElement.asRecord().getValue(); - bucketAssignerFunction.processElement(bootstrapRecord, null, collector); - bucketAssignFunctionContext.setCurrentKey(bootstrapRecord.getRecordKey()); - } - } - - bootstrapOperator.processElement(new StreamRecord<>(hoodieRecord)); - list.clear(); - } - + ScalaCollector> collector = ScalaCollector.getInstance(); bucketAssignerFunction.processElement(hoodieRecord, null, collector); bucketAssignFunctionContext.setCurrentKey(hoodieRecord.getRecordKey()); - writeFunction.processElement(hoodieRecords[0], null, null); + writeFunction.processElement(collector.getVal(), null, null); } public WriteMetadataEvent[] getEventBuffer() { @@ -307,4 +280,26 @@ public boolean isKeyInState(String key) { return this.updateKeys.contains(key); } } + + private static class ScalaCollector implements Collector { + private T val; + + public static ScalaCollector getInstance() { + return new ScalaCollector<>(); + } + + @Override + public void collect(T t) { + this.val = t; + } + + @Override + public void close() { + this.val = null; + } + + public T getVal() { + return val; + } + } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestFunctionWrapper.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestFunctionWrapper.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestFunctionWrapper.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestFunctionWrapper.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestFileIndex.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java similarity index 97% rename from hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index 903be90b964be..72c0890bbf649 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.adapter.TestTableEnvs; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -31,12 +32,10 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.catalog.ObjectPath; @@ -89,24 +88,7 @@ void beforeEach() { execConf.setString("restart-strategy", "fixed-delay"); execConf.setString("restart-strategy.fixed-delay.attempts", "0"); - Configuration conf = new Configuration(); - // for batch upsert use cases: current suggestion is to disable these 2 options, - // from 1.14, flink runtime execution mode has switched from streaming - // to batch for batch execution mode(before that, both streaming and batch use streaming execution mode), - // current batch execution mode has these limitations: - // - // 1. the keyed stream default to always sort the inputs by key; - // 2. the batch state-backend requires the inputs sort by state key - // - // For our hudi batch pipeline upsert case, we rely on the consuming sequence for index records and data records, - // the index records must be loaded first before data records for BucketAssignFunction to keep upsert semantics correct, - // so we suggest disabling these 2 options to use streaming state-backend for batch execution mode - // to keep the strategy before 1.14. - conf.setBoolean("execution.sorted-inputs.enabled", false); - conf.setBoolean("execution.batch-state-backend.enabled", false); - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(conf); - settings = EnvironmentSettings.newInstance().inBatchMode().build(); - batchTableEnv = StreamTableEnvironment.create(execEnv, settings); + batchTableEnv = TestTableEnvs.getBatchTableEnv(); batchTableEnv.getConfig().getConfiguration() .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); } diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableSource.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java similarity index 98% rename from hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java index 67d3f18998326..3930e763fbaaa 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java @@ -101,6 +101,7 @@ public class TestHoodieCatalog { new ResolvedSchema(EXPECTED_TABLE_COLUMNS, Collections.emptyList(), CONSTRAINTS); private static final Map EXPECTED_OPTIONS = new HashMap<>(); + static { EXPECTED_OPTIONS.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ); EXPECTED_OPTIONS.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false"); @@ -243,7 +244,7 @@ public void testGetTable() throws Exception { // validate comment assertEquals(EXPECTED_CATALOG_TABLE.getComment(), actualTable.getComment()); // validate partition key - assertEquals(EXPECTED_CATALOG_TABLE.getPartitionKeys(),((CatalogTable) actualTable).getPartitionKeys()); + assertEquals(EXPECTED_CATALOG_TABLE.getPartitionKeys(), ((CatalogTable) actualTable).getPartitionKeys()); } @Test diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/SchemaBuilder.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/SchemaBuilder.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/SchemaBuilder.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/SchemaBuilder.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestCompactionUtil.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java similarity index 93% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java index d1b6e04a1835d..e1106671799b7 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestConfigurations.java @@ -47,11 +47,11 @@ private TestConfigurations() { } public static final DataType ROW_DATA_TYPE = DataTypes.ROW( - DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key - DataTypes.FIELD("name", DataTypes.VARCHAR(10)), - DataTypes.FIELD("age", DataTypes.INT()), - DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field - DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) + DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key + DataTypes.FIELD("name", DataTypes.VARCHAR(10)), + DataTypes.FIELD("age", DataTypes.INT()), + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field + DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) .notNull(); public static final RowType ROW_TYPE = (RowType) ROW_DATA_TYPE.getLogicalType(); @@ -64,12 +64,12 @@ private TestConfigurations() { .map(RowType.RowField::asSummaryString).collect(Collectors.toList()); public static final DataType ROW_DATA_TYPE_WIDER = DataTypes.ROW( - DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key - DataTypes.FIELD("name", DataTypes.VARCHAR(10)), - DataTypes.FIELD("age", DataTypes.INT()), - DataTypes.FIELD("salary", DataTypes.DOUBLE()), - DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field - DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) + DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key + DataTypes.FIELD("name", DataTypes.VARCHAR(10)), + DataTypes.FIELD("age", DataTypes.INT()), + DataTypes.FIELD("salary", DataTypes.DOUBLE()), + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(3)), // precombine field + DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) .notNull(); public static final RowType ROW_TYPE_WIDER = (RowType) ROW_DATA_TYPE_WIDER.getLogicalType(); diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java similarity index 90% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java index 7729042837a3d..10e7ca1b2c29e 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestHoodieRowData.java @@ -54,17 +54,17 @@ public class TestHoodieRowData { private static final int ROW_INDEX = 10; private static final DataType BASIC_DATA_TYPE = DataTypes.ROW( - DataTypes.FIELD("integer", DataTypes.INT()), - DataTypes.FIELD("string", DataTypes.STRING()), - DataTypes.FIELD("boolean", DataTypes.BOOLEAN()), - DataTypes.FIELD("short", DataTypes.SMALLINT()), - DataTypes.FIELD("byte", DataTypes.TINYINT()), - DataTypes.FIELD("long", DataTypes.BIGINT()), - DataTypes.FIELD("float", DataTypes.FLOAT()), - DataTypes.FIELD("double", DataTypes.DOUBLE()), - DataTypes.FIELD("decimal", DataTypes.DECIMAL(10, 4)), - DataTypes.FIELD("binary", DataTypes.BYTES()), - DataTypes.FIELD("row", DataTypes.ROW())) + DataTypes.FIELD("integer", DataTypes.INT()), + DataTypes.FIELD("string", DataTypes.STRING()), + DataTypes.FIELD("boolean", DataTypes.BOOLEAN()), + DataTypes.FIELD("short", DataTypes.SMALLINT()), + DataTypes.FIELD("byte", DataTypes.TINYINT()), + DataTypes.FIELD("long", DataTypes.BIGINT()), + DataTypes.FIELD("float", DataTypes.FLOAT()), + DataTypes.FIELD("double", DataTypes.DOUBLE()), + DataTypes.FIELD("decimal", DataTypes.DECIMAL(10, 4)), + DataTypes.FIELD("binary", DataTypes.BYTES()), + DataTypes.FIELD("row", DataTypes.ROW())) .notNull(); private static final RowType ROW_TYPE = (RowType) BASIC_DATA_TYPE.getLogicalType(); diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestSQL.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestStringToRowDataConverter.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStringToRowDataConverter.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestStringToRowDataConverter.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStringToRowDataConverter.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/TestViewStorageProperties.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestViewStorageProperties.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/TestViewStorageProperties.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestViewStorageProperties.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/factory/CollectSinkTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/factory/CollectSinkTableFactory.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/factory/CollectSinkTableFactory.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/factory/CollectSinkTableFactory.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/factory/ContinuousFileSourceFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/factory/ContinuousFileSourceFactory.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/factory/ContinuousFileSourceFactory.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/factory/ContinuousFileSourceFactory.java diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java similarity index 100% rename from hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java rename to hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java diff --git a/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink-datasource/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to hudi-flink-datasource/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/hudi-flink/src/test/resources/debezium_json.data b/hudi-flink-datasource/hudi-flink/src/test/resources/debezium_json.data similarity index 100% rename from hudi-flink/src/test/resources/debezium_json.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/debezium_json.data diff --git a/hudi-flink/src/test/resources/log4j-surefire-quiet.properties b/hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire-quiet.properties similarity index 99% rename from hudi-flink/src/test/resources/log4j-surefire-quiet.properties rename to hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire-quiet.properties index 2b94ea2903067..40171af5f4083 100644 --- a/hudi-flink/src/test/resources/log4j-surefire-quiet.properties +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire-quiet.properties @@ -18,7 +18,6 @@ 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. diff --git a/hudi-flink/src/test/resources/log4j-surefire.properties b/hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire.properties similarity index 99% rename from hudi-flink/src/test/resources/log4j-surefire.properties rename to hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire.properties index 8dcd17f303f6b..5806188cb5c58 100644 --- a/hudi-flink/src/test/resources/log4j-surefire.properties +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/log4j-surefire.properties @@ -19,7 +19,6 @@ log4j.rootLogger=INFO, 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. diff --git a/hudi-flink/src/test/resources/test_read_schema.avsc b/hudi-flink-datasource/hudi-flink/src/test/resources/test_read_schema.avsc similarity index 100% rename from hudi-flink/src/test/resources/test_read_schema.avsc rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_read_schema.avsc diff --git a/hudi-flink/src/test/resources/test_source.data b/hudi-flink-datasource/hudi-flink/src/test/resources/test_source.data similarity index 100% rename from hudi-flink/src/test/resources/test_source.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_source.data diff --git a/hudi-flink/src/test/resources/test_source_2.data b/hudi-flink-datasource/hudi-flink/src/test/resources/test_source_2.data similarity index 100% rename from hudi-flink/src/test/resources/test_source_2.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_source_2.data diff --git a/hudi-flink/src/test/resources/test_source_3.data b/hudi-flink-datasource/hudi-flink/src/test/resources/test_source_3.data similarity index 100% rename from hudi-flink/src/test/resources/test_source_3.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_source_3.data diff --git a/hudi-flink/src/test/resources/test_source_4.data b/hudi-flink-datasource/hudi-flink/src/test/resources/test_source_4.data similarity index 100% rename from hudi-flink/src/test/resources/test_source_4.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_source_4.data diff --git a/hudi-flink/src/test/resources/test_source_5.data b/hudi-flink-datasource/hudi-flink/src/test/resources/test_source_5.data similarity index 100% rename from hudi-flink/src/test/resources/test_source_5.data rename to hudi-flink-datasource/hudi-flink/src/test/resources/test_source_5.data diff --git a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml new file mode 100644 index 0000000000000..f6e4f5bc88dfe --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml @@ -0,0 +1,90 @@ + + + + + hudi-flink-datasource + org.apache.hudi + 0.11.0-SNAPSHOT + + 4.0.0 + + hudi-flink1.13.x + 0.11.0-SNAPSHOT + jar + + + ${project.parent.parent.basedir} + + + + + org.apache.flink + flink-table-runtime-blink_${scala.binary.version} + ${flink1.13.version} + provided + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink1.13.version} + provided + + + org.apache.flink + flink-core + ${flink1.13.version} + provided + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink1.13.version} + test + test-jar + + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + + \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java new file mode 100644 index 0000000000000..09da6180d9c63 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarArrayData.java @@ -0,0 +1,262 @@ +/* + * 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.flink.table.data; + +import org.apache.flink.table.data.binary.TypedSetters; +import org.apache.flink.table.data.vector.ArrayColumnVector; +import org.apache.flink.table.data.vector.BooleanColumnVector; +import org.apache.flink.table.data.vector.ByteColumnVector; +import org.apache.flink.table.data.vector.BytesColumnVector; +import org.apache.flink.table.data.vector.ColumnVector; +import org.apache.flink.table.data.vector.DecimalColumnVector; +import org.apache.flink.table.data.vector.DoubleColumnVector; +import org.apache.flink.table.data.vector.FloatColumnVector; +import org.apache.flink.table.data.vector.IntColumnVector; +import org.apache.flink.table.data.vector.LongColumnVector; +import org.apache.flink.table.data.vector.MapColumnVector; +import org.apache.flink.table.data.vector.RowColumnVector; +import org.apache.flink.table.data.vector.ShortColumnVector; +import org.apache.flink.table.data.vector.TimestampColumnVector; + +import java.util.Arrays; + +/** + * Columnar array to support access to vector column data. + * + *

    References {@code org.apache.flink.table.data.ColumnarArrayData} to include FLINK-15390. + */ +public final class ColumnarArrayData implements ArrayData, TypedSetters { + + private final ColumnVector data; + private final int offset; + private final int numElements; + + public ColumnarArrayData(ColumnVector data, int offset, int numElements) { + this.data = data; + this.offset = offset; + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int pos) { + return data.isNullAt(offset + pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean getBoolean(int pos) { + return ((BooleanColumnVector) data).getBoolean(offset + pos); + } + + @Override + public byte getByte(int pos) { + return ((ByteColumnVector) data).getByte(offset + pos); + } + + @Override + public short getShort(int pos) { + return ((ShortColumnVector) data).getShort(offset + pos); + } + + @Override + public int getInt(int pos) { + return ((IntColumnVector) data).getInt(offset + pos); + } + + @Override + public long getLong(int pos) { + return ((LongColumnVector) data).getLong(offset + pos); + } + + @Override + public float getFloat(int pos) { + return ((FloatColumnVector) data).getFloat(offset + pos); + } + + @Override + public double getDouble(int pos) { + return ((DoubleColumnVector) data).getDouble(offset + pos); + } + + @Override + public StringData getString(int pos) { + BytesColumnVector.Bytes byteArray = getByteArray(pos); + return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return ((DecimalColumnVector) data).getDecimal(offset + pos, precision, scale); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return ((TimestampColumnVector) data).getTimestamp(offset + pos, precision); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("RawValueData is not supported."); + } + + @Override + public byte[] getBinary(int pos) { + BytesColumnVector.Bytes byteArray = getByteArray(pos); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + return Arrays.copyOfRange(byteArray.data, byteArray.offset, byteArray.len); + } + } + + @Override + public ArrayData getArray(int pos) { + return ((ArrayColumnVector) data).getArray(offset + pos); + } + + @Override + public MapData getMap(int pos) { + return ((MapColumnVector) data).getMap(offset + pos); + } + + @Override + public RowData getRow(int pos, int numFields) { + return ((RowColumnVector) data).getRow(offset + pos); + } + + @Override + public void setBoolean(int pos, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int pos, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int pos, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int pos, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int pos, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int pos, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int pos, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int pos, DecimalData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int pos, TimestampData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] res = new boolean[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getBoolean(i); + } + return res; + } + + @Override + public byte[] toByteArray() { + byte[] res = new byte[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getByte(i); + } + return res; + } + + @Override + public short[] toShortArray() { + short[] res = new short[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getShort(i); + } + return res; + } + + @Override + public int[] toIntArray() { + int[] res = new int[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getInt(i); + } + return res; + } + + @Override + public long[] toLongArray() { + long[] res = new long[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getLong(i); + } + return res; + } + + @Override + public float[] toFloatArray() { + float[] res = new float[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getFloat(i); + } + return res; + } + + @Override + public double[] toDoubleArray() { + double[] res = new double[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getDouble(i); + } + return res; + } + + private BytesColumnVector.Bytes getByteArray(int pos) { + return ((BytesColumnVector) data).getBytes(offset + pos); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarMapData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarMapData.java new file mode 100644 index 0000000000000..86f4db3954a2c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarMapData.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.flink.table.data; + +import org.apache.flink.table.data.vector.ColumnVector; + +/** + * Columnar map to support access to vector column data. + * + *

    Referenced from flink 1.14.0 {@code org.apache.flink.table.data.ColumnarMapData}. + */ +public final class ColumnarMapData implements MapData { + + private final ColumnVector keyColumnVector; + private final ColumnVector valueColumnVector; + private final int offset; + private final int numElements; + + public ColumnarMapData( + ColumnVector keyColumnVector, + ColumnVector valueColumnVector, + int offset, + int numElements) { + this.keyColumnVector = keyColumnVector; + this.valueColumnVector = valueColumnVector; + this.offset = offset; + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public ArrayData keyArray() { + return new ColumnarArrayData(keyColumnVector, offset, numElements); + } + + @Override + public ArrayData valueArray() { + return new ColumnarArrayData(valueColumnVector, offset, numElements); + } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException( + "ColumnarMapData do not support equals, please compare fields one by one!"); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException( + "ColumnarMapData do not support hashCode, please hash fields one by one!"); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarRowData.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarRowData.java new file mode 100644 index 0000000000000..fbe225878dfc4 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/ColumnarRowData.java @@ -0,0 +1,223 @@ +/* + * 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.flink.table.data; + +import org.apache.flink.table.data.binary.TypedSetters; +import org.apache.flink.table.data.vector.BytesColumnVector.Bytes; +import org.apache.flink.table.data.vector.VectorizedColumnBatch; +import org.apache.flink.types.RowKind; + +/** + * Columnar row to support access to vector column data. + * It is a row view in {@link VectorizedColumnBatch}. + * + *

    References {@code org.apache.flink.table.data.ColumnarRowData} to include FLINK-15390. + */ +public final class ColumnarRowData implements RowData, TypedSetters { + + private RowKind rowKind = RowKind.INSERT; + private VectorizedColumnBatch vectorizedColumnBatch; + private int rowId; + + public ColumnarRowData() { + } + + public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch) { + this(vectorizedColumnBatch, 0); + } + + public ColumnarRowData(VectorizedColumnBatch vectorizedColumnBatch, int rowId) { + this.vectorizedColumnBatch = vectorizedColumnBatch; + this.rowId = rowId; + } + + public void setVectorizedColumnBatch(VectorizedColumnBatch vectorizedColumnBatch) { + this.vectorizedColumnBatch = vectorizedColumnBatch; + this.rowId = 0; + } + + public void setRowId(int rowId) { + this.rowId = rowId; + } + + @Override + public RowKind getRowKind() { + return rowKind; + } + + @Override + public void setRowKind(RowKind kind) { + this.rowKind = kind; + } + + @Override + public int getArity() { + return vectorizedColumnBatch.getArity(); + } + + @Override + public boolean isNullAt(int pos) { + return vectorizedColumnBatch.isNullAt(rowId, pos); + } + + @Override + public boolean getBoolean(int pos) { + return vectorizedColumnBatch.getBoolean(rowId, pos); + } + + @Override + public byte getByte(int pos) { + return vectorizedColumnBatch.getByte(rowId, pos); + } + + @Override + public short getShort(int pos) { + return vectorizedColumnBatch.getShort(rowId, pos); + } + + @Override + public int getInt(int pos) { + return vectorizedColumnBatch.getInt(rowId, pos); + } + + @Override + public long getLong(int pos) { + return vectorizedColumnBatch.getLong(rowId, pos); + } + + @Override + public float getFloat(int pos) { + return vectorizedColumnBatch.getFloat(rowId, pos); + } + + @Override + public double getDouble(int pos) { + return vectorizedColumnBatch.getDouble(rowId, pos); + } + + @Override + public StringData getString(int pos) { + Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); + return StringData.fromBytes(byteArray.data, byteArray.offset, byteArray.len); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return vectorizedColumnBatch.getDecimal(rowId, pos, precision, scale); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return vectorizedColumnBatch.getTimestamp(rowId, pos, precision); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("RawValueData is not supported."); + } + + @Override + public byte[] getBinary(int pos) { + Bytes byteArray = vectorizedColumnBatch.getByteArray(rowId, pos); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + byte[] ret = new byte[byteArray.len]; + System.arraycopy(byteArray.data, byteArray.offset, ret, 0, byteArray.len); + return ret; + } + } + + @Override + public RowData getRow(int pos, int numFields) { + return vectorizedColumnBatch.getRow(rowId, pos); + } + + @Override + public ArrayData getArray(int pos) { + return vectorizedColumnBatch.getArray(rowId, pos); + } + + @Override + public MapData getMap(int pos) { + return vectorizedColumnBatch.getMap(rowId, pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setBoolean(int pos, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int pos, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int pos, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int pos, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int pos, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int pos, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int pos, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int pos, DecimalData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int pos, TimestampData value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException( + "ColumnarRowData do not support equals, please compare fields one by one!"); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException( + "ColumnarRowData do not support hashCode, please hash fields one by one!"); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.java new file mode 100644 index 0000000000000..9cb28b8610a15 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/MapColumnVector.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.flink.table.data.vector; + +import org.apache.flink.table.data.MapData; + +/** + * Map column vector. + */ +public interface MapColumnVector extends ColumnVector { + MapData getMap(int i); +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.java new file mode 100644 index 0000000000000..5fa3031e5174f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/RowColumnVector.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.flink.table.data.vector; + +import org.apache.flink.table.data.ColumnarRowData; + +/** + * Row column vector. + */ +public interface RowColumnVector extends ColumnVector { + ColumnarRowData getRow(int i); +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.java new file mode 100644 index 0000000000000..a0224705bd9f4 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/flink/table/data/vector/VectorizedColumnBatch.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.flink.table.data.vector; + +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.vector.BytesColumnVector.Bytes; + +import java.io.Serializable; +import java.nio.charset.StandardCharsets; + +/** + * A VectorizedColumnBatch is a set of rows, organized with each column as a vector. It is the unit + * of query execution, organized to minimize the cost per row. + * + *

    {@code VectorizedColumnBatch}s are influenced by Apache Hive VectorizedRowBatch. + * + *

    References {@code org.apache.flink.table.data.vector.VectorizedColumnBatch} to include FLINK-15390. + */ +public class VectorizedColumnBatch implements Serializable { + private static final long serialVersionUID = 8180323238728166155L; + + /** + * This number is carefully chosen to minimize overhead and typically allows one + * VectorizedColumnBatch to fit in cache. + */ + public static final int DEFAULT_SIZE = 2048; + + private int numRows; + public final ColumnVector[] columns; + + public VectorizedColumnBatch(ColumnVector[] vectors) { + this.columns = vectors; + } + + public void setNumRows(int numRows) { + this.numRows = numRows; + } + + public int getNumRows() { + return numRows; + } + + public int getArity() { + return columns.length; + } + + public boolean isNullAt(int rowId, int colId) { + return columns[colId].isNullAt(rowId); + } + + public boolean getBoolean(int rowId, int colId) { + return ((BooleanColumnVector) columns[colId]).getBoolean(rowId); + } + + public byte getByte(int rowId, int colId) { + return ((ByteColumnVector) columns[colId]).getByte(rowId); + } + + public short getShort(int rowId, int colId) { + return ((ShortColumnVector) columns[colId]).getShort(rowId); + } + + public int getInt(int rowId, int colId) { + return ((IntColumnVector) columns[colId]).getInt(rowId); + } + + public long getLong(int rowId, int colId) { + return ((LongColumnVector) columns[colId]).getLong(rowId); + } + + public float getFloat(int rowId, int colId) { + return ((FloatColumnVector) columns[colId]).getFloat(rowId); + } + + public double getDouble(int rowId, int colId) { + return ((DoubleColumnVector) columns[colId]).getDouble(rowId); + } + + public Bytes getByteArray(int rowId, int colId) { + return ((BytesColumnVector) columns[colId]).getBytes(rowId); + } + + private byte[] getBytes(int rowId, int colId) { + Bytes byteArray = getByteArray(rowId, colId); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + return byteArray.getBytes(); + } + } + + public String getString(int rowId, int colId) { + Bytes byteArray = getByteArray(rowId, colId); + return new String(byteArray.data, byteArray.offset, byteArray.len, StandardCharsets.UTF_8); + } + + public DecimalData getDecimal(int rowId, int colId, int precision, int scale) { + return ((DecimalColumnVector) (columns[colId])).getDecimal(rowId, precision, scale); + } + + public TimestampData getTimestamp(int rowId, int colId, int precision) { + return ((TimestampColumnVector) (columns[colId])).getTimestamp(rowId, precision); + } + + public ArrayData getArray(int rowId, int colId) { + return ((ArrayColumnVector) columns[colId]).getArray(rowId); + } + + public RowData getRow(int rowId, int colId) { + return ((RowColumnVector) columns[colId]).getRow(rowId); + } + + public MapData getMap(int rowId, int colId) { + return ((MapColumnVector) columns[colId]).getMap(rowId); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java new file mode 100644 index 0000000000000..51c53f368fb9d --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** + * Adapter clazz for {@code AbstractStreamOperator}. + */ +public abstract class AbstractStreamOperatorAdapter extends AbstractStreamOperator { + @Override + public void close() throws Exception { + super.dispose(); + } + + public void finish() throws Exception { + super.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java new file mode 100644 index 0000000000000..0ea0968f17585 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.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.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.MailboxExecutor; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Adapter clazz for {@link AbstractStreamOperatorFactory}. + */ +public abstract class AbstractStreamOperatorFactoryAdapter + extends AbstractStreamOperatorFactory implements YieldingOperatorFactory { + private transient MailboxExecutor mailboxExecutor; + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + public MailboxExecutorAdapter getMailboxExecutorAdapter() { + return new MailboxExecutorAdapter(getMailboxExecutor()); + } + + /** + * Provides the mailbox executor iff this factory implements {@link YieldingOperatorFactory}. + */ + protected MailboxExecutor getMailboxExecutor() { + return checkNotNull( + mailboxExecutor, "Factory does not implement %s", YieldingOperatorFactory.class); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java new file mode 100644 index 0000000000000..9ae3ca6912f65 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java @@ -0,0 +1,37 @@ +/* + * 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.adapter; + +import org.apache.flink.streaming.api.operators.MailboxExecutor; +import org.apache.flink.util.function.ThrowingRunnable; + +/** + * Adapter clazz for {@link MailboxExecutor}. + */ +public class MailboxExecutorAdapter { + private final MailboxExecutor executor; + + public MailboxExecutorAdapter(MailboxExecutor executor) { + this.executor = executor; + } + + public void execute(ThrowingRunnable command, String description) { + this.executor.execute(command, description); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java new file mode 100644 index 0000000000000..6d058de89bc55 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java @@ -0,0 +1,40 @@ +/* + * 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.adapter; + +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.RateLimiter; + +/** + * Bridge class for shaded guava clazz {@code RateLimiter}. + */ +public class RateLimiterAdapter { + private final RateLimiter rateLimiter; + + private RateLimiterAdapter(double permitsPerSecond) { + this.rateLimiter = RateLimiter.create(permitsPerSecond); + } + + public static RateLimiterAdapter create(double permitsPerSecond) { + return new RateLimiterAdapter(permitsPerSecond); + } + + public void acquire() { + this.rateLimiter.acquire(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.java new file mode 100644 index 0000000000000..9eb52c6765896 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/Utils.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.adapter; + +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; + +/** + * Adapter utils. + */ +public class Utils { + public static SourceFunction.SourceContext getSourceContext( + TimeCharacteristic timeCharacteristic, + ProcessingTimeService processingTimeService, + StreamTask streamTask, + Output> output, + long watermarkInterval) { + return StreamSourceContexts.getSourceContext( + timeCharacteristic, + processingTimeService, + new Object(), // no actual locking needed + streamTask.getStreamStatusMaintainer(), + output, + watermarkInterval, + -1); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java new file mode 100644 index 0000000000000..18686b811c400 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java @@ -0,0 +1,27 @@ +/* + * 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.adapter; + +import org.apache.flink.streaming.api.operators.Output; + +/** + * Adapter clazz for {@link Output}. + */ +public interface OutputAdapter extends Output { +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java new file mode 100644 index 0000000000000..8563d2422b648 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.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.adapter; + +import org.apache.flink.runtime.state.StateInitializationContext; + +/** + * Adapter clazz for {@link StateInitializationContext}. + */ +public interface StateInitializationContextAdapter extends StateInitializationContext { +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java new file mode 100644 index 0000000000000..176783e8108c6 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.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.adapter; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; + +import java.util.Map; + +/** + * Adapter clazz for {@link StreamingRuntimeContext}. + */ +public class StreamingRuntimeContextAdapter extends StreamingRuntimeContext { + + public StreamingRuntimeContextAdapter(AbstractStreamOperator operator, Environment env, + Map> accumulators) { + super(operator, env, accumulators); + } + + @Override + public MetricGroup getMetricGroup() { + return new UnregisteredMetricsGroup(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java new file mode 100644 index 0000000000000..e3088356709f1 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java @@ -0,0 +1,34 @@ +package org.apache.hudi.adapter; + +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; + +/* + * 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. + */ + +/** + * TableEnv for test goals. + */ +public class TestTableEnvs { + + public static TableEnvironment getBatchTableEnv() { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + return TableEnvironmentImpl.create(settings); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml new file mode 100644 index 0000000000000..0e5df91b49c90 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml @@ -0,0 +1,102 @@ + + + + + hudi-flink-datasource + org.apache.hudi + 0.11.0-SNAPSHOT + + 4.0.0 + + hudi-flink1.14.x + 0.11.0-SNAPSHOT + jar + + + ${project.parent.parent.basedir} + + + + + org.apache.flink + flink-table-api-java + ${flink1.14.version} + provided + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink1.14.version} + provided + + + org.apache.flink + flink-shaded-guava + 30.1.1-jre-14.0 + provided + + + org.apache.flink + flink-core + ${flink1.14.version} + provided + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink1.14.version} + provided + + + org.apache.flink + flink-runtime + ${flink1.14.version} + test + test-jar + + + + + + + org.jacoco + jacoco-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + test-compile + + + + false + + + + org.apache.rat + apache-rat-plugin + + + + \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java new file mode 100644 index 0000000000000..d4c6bc3a8f4da --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorAdapter.java @@ -0,0 +1,27 @@ +/* + * 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.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; + +/** + * Adapter clazz for {@code AbstractStreamOperator}. + */ +public abstract class AbstractStreamOperatorAdapter extends AbstractStreamOperator { +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.java new file mode 100644 index 0000000000000..6dcfe71ccfd9d --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/AbstractStreamOperatorFactoryAdapter.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.adapter; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; + +/** + * Adapter clazz for {@link AbstractStreamOperatorFactory}. + */ +public abstract class AbstractStreamOperatorFactoryAdapter + extends AbstractStreamOperatorFactory implements YieldingOperatorFactory { + + public MailboxExecutorAdapter getMailboxExecutorAdapter() { + return new MailboxExecutorAdapter(getMailboxExecutor()); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java new file mode 100644 index 0000000000000..0c836f3db391b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MailboxExecutorAdapter.java @@ -0,0 +1,37 @@ +/* + * 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.adapter; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.util.function.ThrowingRunnable; + +/** + * Adapter clazz for {@link MailboxExecutor}. + */ +public class MailboxExecutorAdapter { + private final MailboxExecutor executor; + + public MailboxExecutorAdapter(MailboxExecutor executor) { + this.executor = executor; + } + + public void execute(ThrowingRunnable command, String description) { + this.executor.execute(command, description); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java new file mode 100644 index 0000000000000..865c0c81d4d9d --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/RateLimiterAdapter.java @@ -0,0 +1,40 @@ +/* + * 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.adapter; + +import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter; + +/** + * Bridge class for shaded guava clazz {@code RateLimiter}. + */ +public class RateLimiterAdapter { + private final RateLimiter rateLimiter; + + private RateLimiterAdapter(double permitsPerSecond) { + this.rateLimiter = RateLimiter.create(permitsPerSecond); + } + + public static RateLimiterAdapter create(double permitsPerSecond) { + return new RateLimiterAdapter(permitsPerSecond); + } + + public void acquire() { + this.rateLimiter.acquire(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.java new file mode 100644 index 0000000000000..41ac0ffcee576 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/Utils.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.adapter; + +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; + +/** + * Adapter utils. + */ +public class Utils { + public static SourceFunction.SourceContext getSourceContext( + TimeCharacteristic timeCharacteristic, + ProcessingTimeService processingTimeService, + StreamTask streamTask, + Output> output, + long watermarkInterval) { + return StreamSourceContexts.getSourceContext( + timeCharacteristic, + processingTimeService, + new Object(), // no actual locking needed + output, + watermarkInterval, + -1, + true); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.java new file mode 100644 index 0000000000000..c0d83e6096e3c --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/OutputAdapter.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.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; + +/** + * Adapter clazz for {@link Output}. + */ +public interface OutputAdapter extends Output { + @Override + default void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // no operation + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.java new file mode 100644 index 0000000000000..1f76ad692f33f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StateInitializationContextAdapter.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.adapter; + +import org.apache.flink.runtime.state.StateInitializationContext; + +import java.util.OptionalLong; + +/** + * Adapter clazz for {@link StateInitializationContext}. + */ +public interface StateInitializationContextAdapter extends StateInitializationContext { + @Override + default OptionalLong getRestoredCheckpointId() { + return OptionalLong.empty(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.java new file mode 100644 index 0000000000000..4461c28943d3a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/StreamingRuntimeContextAdapter.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.adapter; + +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.metrics.groups.OperatorMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; + +import java.util.Map; + +/** + * Adapter clazz for {@link StreamingRuntimeContext}. + */ +public class StreamingRuntimeContextAdapter extends StreamingRuntimeContext { + + public StreamingRuntimeContextAdapter(AbstractStreamOperator operator, Environment env, + Map> accumulators) { + super(operator, env, accumulators); + } + + @Override + public OperatorMetricGroup getMetricGroup() { + return UnregisteredMetricsGroup.createOperatorMetricGroup(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.java new file mode 100644 index 0000000000000..e65437609a21e --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/test/java/org/apache/hudi/adapter/TestTableEnvs.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.adapter; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +/** + * TableEnv for test goals. + */ +public class TestTableEnvs { + + public static TableEnvironment getBatchTableEnv() { + Configuration conf = new Configuration(); + // for batch upsert use cases: current suggestion is to disable these 2 options, + // from 1.14, flink runtime execution mode has switched from streaming + // to batch for batch execution mode(before that, both streaming and batch use streaming execution mode), + // current batch execution mode has these limitations: + // + // 1. the keyed stream default to always sort the inputs by key; + // 2. the batch state-backend requires the inputs sort by state key + // + // For our hudi batch pipeline upsert case, we rely on the consuming sequence for index records and data records, + // the index records must be loaded first before data records for BucketAssignFunction to keep upsert semantics correct, + // so we suggest disabling these 2 options to use streaming state-backend for batch execution mode + // to keep the strategy before 1.14. + conf.setBoolean("execution.sorted-inputs.enabled", false); + conf.setBoolean("execution.batch-state-backend.enabled", false); + StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(conf); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + return StreamTableEnvironment.create(execEnv, settings); + } +} diff --git a/hudi-flink-datasource/pom.xml b/hudi-flink-datasource/pom.xml new file mode 100644 index 0000000000000..1ce7735b9844a --- /dev/null +++ b/hudi-flink-datasource/pom.xml @@ -0,0 +1,41 @@ + + + + + hudi + org.apache.hudi + 0.11.0-SNAPSHOT + + 4.0.0 + + hudi-flink-datasource + 0.11.0-SNAPSHOT + pom + + + ${project.parent.basedir} + + + + hudi-flink1.13.x + hudi-flink1.14.x + hudi-flink + + + diff --git a/hudi-flink/pom.xml b/hudi-flink/pom.xml deleted file mode 100644 index 7338b23bf4b27..0000000000000 --- a/hudi-flink/pom.xml +++ /dev/null @@ -1,349 +0,0 @@ - - - - - - - hudi - org.apache.hudi - 0.11.0-SNAPSHOT - - 4.0.0 - - hudi-flink_${scala.binary.version} - jar - - - ${project.parent.basedir} - 1.11.1 - - - - - - org.jacoco - jacoco-maven-plugin - - - org.apache.maven.plugins - maven-compiler-plugin - - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-jar-plugin - 3.1.2 - - - - test-jar - - - - - - org.apache.rat - apache-rat-plugin - - - - - - src/main/resources - - - src/test/resources - - - - - - - - org.apache.hudi - hudi-common - ${project.version} - - - org.apache.hudi - hudi-client-common - ${project.version} - - - org.apache.hudi - hudi-flink-client - ${project.version} - - - org.apache.hudi - hudi-hadoop-mr - ${project.version} - - - org.apache.hudi - hudi-hive-sync - ${project.version} - - - org.apache.hudi - hudi-sync-common - ${project.version} - - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - compile - - - org.apache.flink - flink-clients_${scala.binary.version} - compile - - - com.esotericsoftware.kryo - kryo - - - com.esotericsoftware.minlog - minlog - - - - - org.apache.flink - flink-connector-kafka_${scala.binary.version} - compile - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - org.apache.flink - flink-hadoop-compatibility_${scala.binary.version} - ${flink.version} - - - org.apache.flink - flink-parquet_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-json - ${flink.version} - provided - - - org.apache.flink - flink-table-common - ${flink.version} - provided - - - org.apache.flink - flink-table-runtime_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${flink.version} - provided - - - org.apache.flink - flink-statebackend-rocksdb_${scala.binary.version} - ${flink.version} - provided - - - - org.apache.parquet - parquet-hadoop - ${parquet.version} - - - org.xerial.snappy - snappy-java - - - - - - - org.apache.parquet - parquet-avro - ${parquet.version} - test - - - - - org.apache.avro - avro - - 1.10.0 - compile - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - compile - - - org.slf4j - slf4j-log4j12 - - - - - - com.beust - jcommander - compile - - - com.twitter - bijection-avro_${scala.binary.version} - 0.9.7 - - - joda-time - joda-time - 2.5 - - - - ${hive.groupid} - hive-exec - ${hive.version} - ${hive.exec.classifier} - - - javax.mail - mail - - - org.eclipse.jetty.aggregate - * - - - - - - - - - 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.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.flink - flink-test-utils_${scala.binary.version} - ${flink.version} - test - - - org.apache.flink - flink-runtime - ${flink.version} - test - test-jar - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink.version} - test - test-jar - - - org.apache.flink - flink-table-runtime_${scala.binary.version} - ${flink.version} - test - test-jar - - - org.apache.flink - flink-json - ${flink.version} - test - test-jar - - - org.apache.flink - flink-csv - ${flink.version} - test - - - diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml index 8845bfb801ae3..961855b034867 100644 --- a/hudi-kafka-connect/pom.xml +++ b/hudi-kafka-connect/pom.xml @@ -142,7 +142,7 @@ org.apache.hudi - hudi-flink_${scala.binary.version} + hudi-flink ${project.version} diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 222478090b4b0..30ee37a4ecf2f 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -25,7 +25,7 @@ ../../pom.xml 4.0.0 - hudi-flink-bundle_${scala.binary.version} + hudi-flink${flink.bundle.version}-bundle_${scala.binary.version} jar @@ -76,7 +76,8 @@ org.apache.hudi:hudi-common org.apache.hudi:hudi-client-common org.apache.hudi:hudi-flink-client - org.apache.hudi:hudi-flink_${scala.binary.version} + org.apache.hudi:hudi-flink + org.apache.hudi:${hudi.flink.module} org.apache.hudi:hudi-hive-sync org.apache.hudi:hudi-sync-common org.apache.hudi:hudi-hadoop-mr @@ -254,7 +255,12 @@ org.apache.hudi - hudi-flink_${scala.binary.version} + ${hudi.flink.module} + ${project.version} + + + org.apache.hudi + hudi-flink ${project.version} diff --git a/pom.xml b/pom.xml index 5be49eb04c180..c61d5ef8f3a75 100644 --- a/pom.xml +++ b/pom.xml @@ -52,7 +52,7 @@ packaging/hudi-timeline-server-bundle packaging/hudi-trino-bundle hudi-examples - hudi-flink + hudi-flink-datasource hudi-kafka-connect packaging/hudi-flink-bundle packaging/hudi-kafka-connect-bundle @@ -113,10 +113,17 @@ 0.8.0 4.4.1 ${spark2.version} - - 1.14.3 2.4.4 3.2.1 + + 1.14.4 + 1.13.6 + ${flink1.14.version} + hudi-flink1.14.x + 1.14 + flink-runtime + flink-table-runtime_${scala.binary.version} + flink-table-planner_${scala.binary.version} hudi-spark2 hudi-spark2-common 1.8.2 @@ -1696,6 +1703,35 @@ + + flink1.14 + + true + + flink1.14 + + !disabled + + + + + flink1.13 + + ${flink1.13.version} + flink-runtime_${scala.binary.version} + flink-table-runtime-blink_${scala.binary.version} + flink-table-planner-blink_${scala.binary.version} + hudi-flink1.13.x + 1.13 + true + + + + flink1.13 + + + + skipShadeSources From a118d56b071bad6a4701c3578e43e5fdfe416e6c Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Sun, 20 Mar 2022 21:56:30 -0700 Subject: [PATCH 29/33] [MINOR] Fixing sparkUpdateNode for record generation (#5079) --- .../hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala | 6 +++++- .../hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala | 5 +++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala index b8c46cad3fd69..b0bec48a40e83 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala @@ -47,7 +47,7 @@ class SparkInsertNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { override def execute(context: ExecutionContext, curItrCount: Int): Unit = { if (!config.isDisableGenerate) { println("Generating input data for node {}", this.getName) - context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + writeRecords(context) } val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, context.getWriterContext.getHoodieTestSuiteWriter.getSchema, @@ -68,4 +68,8 @@ class SparkInsertNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { def getOperation(): String = { DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL } + + def writeRecords(context: ExecutionContext): Unit = { + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count() + } } diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala index 113de93adbb3a..f83bc55633db2 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala @@ -20,6 +20,7 @@ package org.apache.hudi.integ.testsuite.dag.nodes import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext /** * Spark datasource based upsert node @@ -31,4 +32,8 @@ class SparkUpsertNode(dagNodeConfig: Config) extends SparkInsertNode(dagNodeConf override def getOperation(): String = { DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL } + + override def writeRecords(context: ExecutionContext): Unit = { + context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateUpdates(config)).count() + } } From 26e5d2e6fcc399e7886594a5454a16d29aaa8702 Mon Sep 17 00:00:00 2001 From: wxp4532 Date: Fri, 11 Mar 2022 14:07:52 +0800 Subject: [PATCH 30/33] [HUDI-3559] Flink bucket index with COW table throws NoSuchElementException Actually method FlinkWriteHelper#deduplicateRecords does not guarantee the records sequence, but there is a implicit constraint: all the records in one bucket should have the same bucket type(instant time here), the BucketStreamWriteFunction breaks the rule and fails to comply with this constraint. close apache/hudi#5018 --- .../table/action/commit/FlinkWriteHelper.java | 11 +- .../hudi/sink/BucketStreamWriteFunction.java | 32 +++- .../hudi/sink/ITTestDataStreamWrite.java | 158 ++++++++++-------- 3 files changed, 113 insertions(+), 88 deletions(-) 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 d28aafcc4abf8..66723a3fcdb7a 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 @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; @@ -91,13 +90,11 @@ protected List> tag(List> dedupedRecords, Hoodie @Override public List> deduplicateRecords( List> records, HoodieIndex index, int parallelism) { - Map>>> keyedRecords = records.stream().map(record -> { - // If index used is global, then records are expected to differ in their partitionPath - final Object key = record.getKey().getRecordKey(); - return Pair.of(key, record); - }).collect(Collectors.groupingBy(Pair::getLeft)); + // If index used is global, then records are expected to differ in their partitionPath + Map>> keyedRecords = records.stream() + .collect(Collectors.groupingBy(record -> record.getKey().getRecordKey())); - return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> { + return keyedRecords.values().stream().map(x -> x.stream().reduce((rec1, rec2) -> { final T data1 = rec1.getData(); final T data2 = rec2.getData(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java index 057c794331d66..4c9e4dc25912b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/BucketStreamWriteFunction.java @@ -65,7 +65,17 @@ public class BucketStreamWriteFunction extends StreamWriteFunction { private String indexKeyFields; - private final HashMap bucketToFileIDMap; + /** + * BucketID to file group mapping. + */ + private HashMap bucketIndex; + + /** + * Incremental bucket index of the current checkpoint interval, + * it is needed because the bucket type('I' or 'U') should be decided based on the committed files view, + * all the records in one bucket should have the same bucket type. + */ + private HashMap incBucketIndex; /** * Constructs a BucketStreamWriteFunction. @@ -74,7 +84,6 @@ public class BucketStreamWriteFunction extends StreamWriteFunction { */ public BucketStreamWriteFunction(Configuration config) { super(config); - this.bucketToFileIDMap = new HashMap<>(); } @Override @@ -85,6 +94,8 @@ public void open(Configuration parameters) throws IOException { this.taskID = getRuntimeContext().getIndexOfThisSubtask(); this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); this.maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks(); + this.bucketIndex = new HashMap<>(); + this.incBucketIndex = new HashMap<>(); bootstrapIndex(); } @@ -94,6 +105,13 @@ public void initializeState(FunctionInitializationContext context) throws Except this.table = this.writeClient.getHoodieTable(); } + @Override + public void snapshotState() { + super.snapshotState(); + this.bucketIndex.putAll(this.incBucketIndex); + this.incBucketIndex.clear(); + } + @Override public void processElement(I i, ProcessFunction.Context context, Collector collector) throws Exception { HoodieRecord record = (HoodieRecord) i; @@ -103,12 +121,12 @@ public void processElement(I i, ProcessFunction.Context context, Coll final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, this.bucketNum); final String partitionBucketId = BucketIdentifier.partitionBucketIdStr(hoodieKey.getPartitionPath(), bucketNum); - if (bucketToFileIDMap.containsKey(partitionBucketId)) { - location = new HoodieRecordLocation("U", bucketToFileIDMap.get(partitionBucketId)); + if (bucketIndex.containsKey(partitionBucketId)) { + location = new HoodieRecordLocation("U", bucketIndex.get(partitionBucketId)); } else { String newFileId = BucketIdentifier.newBucketFileIdPrefix(bucketNum); location = new HoodieRecordLocation("I", newFileId); - bucketToFileIDMap.put(partitionBucketId, newFileId); + incBucketIndex.put(partitionBucketId, newFileId); } record.unseal(); record.setCurrentLocation(location); @@ -154,12 +172,12 @@ private void bootstrapIndex() throws IOException { if (bucketToLoad.contains(bucketNumber)) { String partitionBucketId = BucketIdentifier.partitionBucketIdStr(partitionPath, bucketNumber); LOG.info(String.format("Should load this partition bucket %s with fileID %s", partitionBucketId, fileID)); - if (bucketToFileIDMap.containsKey(partitionBucketId)) { + if (bucketIndex.containsKey(partitionBucketId)) { throw new RuntimeException(String.format("Duplicate fileID %s from partitionBucket %s found " + "during the BucketStreamWriteFunction index bootstrap.", fileID, partitionBucketId)); } else { LOG.info(String.format("Adding fileID %s to the partition bucket %s.", fileID, partitionBucketId)); - bucketToFileIDMap.put(partitionBucketId, fileID); + bucketIndex.put(partitionBucketId, fileID); } } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java index bbf9009fd5b75..b9deb43a97c53 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.transform.ChainedTransformer; import org.apache.hudi.sink.transform.Transformer; @@ -92,8 +93,20 @@ public class ITTestDataStreamWrite extends TestLogger { @TempDir File tempFile; + @ParameterizedTest + @ValueSource(strings = {"BUCKET", "FLINK_STATE"}) + public void testWriteCopyOnWrite(String indexType) throws Exception { + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + conf.setString(FlinkOptions.INDEX_TYPE, indexType); + conf.setInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS, 1); + conf.setString(FlinkOptions.INDEX_KEY_FIELD, "id"); + conf.setBoolean(FlinkOptions.PRE_COMBINE,true); + + testWriteToHoodie(conf, "cow_write", 1, EXPECTED); + } + @Test - public void testTransformerBeforeWriting() throws Exception { + public void testWriteCopyOnWriteWithTransformer() throws Exception { Transformer transformer = (ds) -> ds.map((rowdata) -> { if (rowdata instanceof GenericRowData) { GenericRowData genericRD = (GenericRowData) rowdata; @@ -105,97 +118,63 @@ public void testTransformerBeforeWriting() throws Exception { } }); - testWriteToHoodie(transformer, EXPECTED_TRANSFORMER); + testWriteToHoodie(transformer, "cow_write_with_transformer", EXPECTED_TRANSFORMER); } @Test - public void testChainedTransformersBeforeWriting() throws Exception { - Transformer t1 = (ds) -> ds.map((rowdata) -> { - if (rowdata instanceof GenericRowData) { - GenericRowData genericRD = (GenericRowData) rowdata; + public void testWriteCopyOnWriteWithChainedTransformer() throws Exception { + Transformer t1 = (ds) -> ds.map(rowData -> { + if (rowData instanceof GenericRowData) { + GenericRowData genericRD = (GenericRowData) rowData; //update age field to age + 1 genericRD.setField(2, genericRD.getInt(2) + 1); return genericRD; } else { - throw new RuntimeException("Unrecognized row type : " + rowdata.getClass().getSimpleName()); + throw new RuntimeException("Unrecognized row type : " + rowData.getClass().getSimpleName()); } }); ChainedTransformer chainedTransformer = new ChainedTransformer(Arrays.asList(t1, t1)); - testWriteToHoodie(chainedTransformer, EXPECTED_CHAINED_TRANSFORMER); - } - - @Test - public void testWriteToHoodieWithoutTransformer() throws Exception { - testWriteToHoodie(null, EXPECTED); + testWriteToHoodie(chainedTransformer, "cow_write_with_chained_transformer", EXPECTED_CHAINED_TRANSFORMER); } @ParameterizedTest @ValueSource(strings = {"BUCKET", "FLINK_STATE"}) - public void testMergeOnReadWriteWithCompaction(String indexType) throws Exception { - int parallelism = 4; + public void testWriteMergeOnReadWithCompaction(String indexType) throws Exception { Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); conf.setString(FlinkOptions.INDEX_TYPE, indexType); conf.setInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS, 4); conf.setString(FlinkOptions.INDEX_KEY_FIELD, "id"); conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1); conf.setString(FlinkOptions.TABLE_TYPE, HoodieTableType.MERGE_ON_READ.name()); - StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - execEnv.getConfig().disableObjectReuse(); - execEnv.setParallelism(parallelism); - // set up checkpoint interval - execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE); - execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1); - // Read from file source - RowType rowType = - (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) - .getLogicalType(); - - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601 - ); - String sourcePath = Objects.requireNonNull(Thread.currentThread() - .getContextClassLoader().getResource("test_source.data")).toString(); - - TextInputFormat format = new TextInputFormat(new Path(sourcePath)); - format.setFilesFilter(FilePathFilter.createDefaultFilter()); - TypeInformation typeInfo = BasicTypeInfo.STRING_TYPE_INFO; - format.setCharsetName("UTF-8"); - - DataStream dataStream = execEnv - // use PROCESS_CONTINUOUSLY mode to trigger checkpoint - .readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) - .setParallelism(parallelism); + testWriteToHoodie(conf, "mor_write_with_compact", 1, EXPECTED); + } - DataStream hoodieRecordDataStream = Pipelines.bootstrap(conf, rowType, parallelism, dataStream); - DataStream pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream); - Pipelines.clean(conf, pipeline); - Pipelines.compact(conf, pipeline); - JobClient client = execEnv.executeAsync("mor-write-with-compact"); - if (client.getJobStatus().get() != JobStatus.FAILED) { - try { - TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish - client.cancel(); - } catch (Throwable var1) { - // ignored - } - } + private void testWriteToHoodie( + Transformer transformer, + String jobName, + Map> expected) throws Exception { + testWriteToHoodie(TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), + Option.of(transformer), jobName, 2, expected); + } - TestData.checkWrittenFullData(tempFile, EXPECTED); + private void testWriteToHoodie( + Configuration conf, + String jobName, + int checkpoints, + Map> expected) throws Exception { + testWriteToHoodie(conf, Option.empty(), jobName, checkpoints, expected); } private void testWriteToHoodie( - Transformer transformer, + Configuration conf, + Option transformer, + String jobName, + int checkpoints, Map> expected) throws Exception { - Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); execEnv.getConfig().disableObjectReuse(); execEnv.setParallelism(4); @@ -218,16 +197,32 @@ private void testWriteToHoodie( String sourcePath = Objects.requireNonNull(Thread.currentThread() .getContextClassLoader().getResource("test_source.data")).toString(); - DataStream dataStream = execEnv - // use continuous file source to trigger checkpoint - .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), 2)) - .name("continuous_file_source") - .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) - .setParallelism(4); + boolean isMor = conf.getString(FlinkOptions.TABLE_TYPE).equals(HoodieTableType.MERGE_ON_READ.name()); + + DataStream dataStream; + if (isMor) { + TextInputFormat format = new TextInputFormat(new Path(sourcePath)); + format.setFilesFilter(FilePathFilter.createDefaultFilter()); + TypeInformation typeInfo = BasicTypeInfo.STRING_TYPE_INFO; + format.setCharsetName("UTF-8"); + + dataStream = execEnv + // use PROCESS_CONTINUOUSLY mode to trigger checkpoint + .readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo) + .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .setParallelism(1); + } else { + dataStream = execEnv + // use continuous file source to trigger checkpoint + .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), checkpoints)) + .name("continuous_file_source") + .setParallelism(1) + .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .setParallelism(4); + } - if (transformer != null) { - dataStream = transformer.apply(dataStream); + if (transformer.isPresent()) { + dataStream = transformer.get().apply(dataStream); } int parallelism = execEnv.getParallelism(); @@ -235,9 +230,24 @@ private void testWriteToHoodie( DataStream pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream); execEnv.addOperator(pipeline.getTransformation()); - JobClient client = execEnv.executeAsync(conf.getString(FlinkOptions.TABLE_NAME)); - // wait for the streaming job to finish - client.getJobExecutionResult().get(); + if (isMor) { + Pipelines.clean(conf, pipeline); + Pipelines.compact(conf, pipeline); + } + JobClient client = execEnv.executeAsync(jobName); + if (isMor) { + if (client.getJobStatus().get() != JobStatus.FAILED) { + try { + TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish + client.cancel(); + } catch (Throwable var1) { + // ignored + } + } + } else { + // wait for the streaming job to finish + client.getJobExecutionResult().get(); + } TestData.checkWrittenFullData(tempFile, expected); From ca0931d332234d0b743b4a035901a3bc9325d47c Mon Sep 17 00:00:00 2001 From: Pratyaksh Sharma Date: Tue, 22 Mar 2022 05:36:30 +0530 Subject: [PATCH 31/33] [HUDI-1436]: Provide an option to trigger clean every nth commit (#4385) - Provided option to trigger clean every nth commit with default number of commits as 1 so that existing users are not affected. Co-authored-by: sivabalan --- .../hudi/config/HoodieCompactionConfig.java | 22 + .../apache/hudi/config/HoodieWriteConfig.java | 17 +- .../action/clean/CleanPlanActionExecutor.java | 30 +- .../action/clean/CleaningTriggerStrategy.java | 24 ++ .../action/commit/TestHoodieConcatHandle.java | 8 +- .../TestJavaCopyOnWriteActionExecutor.java | 4 +- .../org/apache/hudi/table/TestCleaner.java | 391 ++--------------- .../functional/TestCleanPlanExecutor.java | 397 +++++++++++++++++- .../common/testutils/HoodieTestTable.java | 12 +- 9 files changed, 521 insertions(+), 384 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleaningTriggerStrategy.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index 2570e204eef3a..4003a07de7f0b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy; @@ -129,6 +130,17 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("Controls how compaction scheduling is triggered, by time or num delta commits or combination of both. " + "Valid options: " + Arrays.stream(CompactionTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); + public static final ConfigProperty CLEAN_TRIGGER_STRATEGY = ConfigProperty + .key("hoodie.clean.trigger.strategy") + .defaultValue(CleaningTriggerStrategy.NUM_COMMITS.name()) + .withDocumentation("Controls how cleaning is scheduled. Valid options: " + + Arrays.stream(CleaningTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(","))); + + public static final ConfigProperty CLEAN_MAX_COMMITS = ConfigProperty + .key("hoodie.clean.max.commits") + .defaultValue("1") + .withDocumentation("Number of commits after the last clean operation, before scheduling of a new clean is attempted."); + public static final ConfigProperty CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty .key("hoodie.cleaner.fileversions.retained") .defaultValue("3") @@ -583,6 +595,16 @@ public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy com return this; } + public Builder withCleaningTriggerStrategy(String cleaningTriggerStrategy) { + compactionConfig.setValue(CLEAN_TRIGGER_STRATEGY, cleaningTriggerStrategy); + return this; + } + + public Builder withMaxCommitsBeforeCleaning(int maxCommitsBeforeCleaning) { + compactionConfig.setValue(CLEAN_MAX_COMMITS, String.valueOf(maxCommitsBeforeCleaning)); + return this; + } + public Builder withCleanerPolicy(HoodieCleaningPolicy policy) { compactionConfig.setValue(CLEANER_POLICY, policy.name()); return this; 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 b2e8b999be513..4847d9c91ed44 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 @@ -61,6 +61,7 @@ import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.table.RandomFileIdPrefixProvider; +import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; @@ -1153,6 +1154,18 @@ public int getCleanerParallelism() { return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM_VALUE); } + public int getCleaningMaxCommits() { + return getInt(HoodieCompactionConfig.CLEAN_MAX_COMMITS); + } + + public CleaningTriggerStrategy getCleaningTriggerStrategy() { + return CleaningTriggerStrategy.valueOf(getString(HoodieCompactionConfig.CLEAN_TRIGGER_STRATEGY)); + } + + public boolean isAutoClean() { + return getBoolean(HoodieCompactionConfig.AUTO_CLEAN); + } + public boolean getArchiveMergeEnable() { return getBoolean(HoodieCompactionConfig.ARCHIVE_MERGE_ENABLE); } @@ -1169,10 +1182,6 @@ public boolean isAsyncArchive() { return getBoolean(HoodieCompactionConfig.ASYNC_ARCHIVE); } - public boolean isAutoClean() { - return getBoolean(HoodieCompactionConfig.AUTO_CLEAN); - } - public boolean isAsyncClean() { return getBoolean(HoodieCompactionConfig.ASYNC_CLEAN); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java index a64cb88454b0c..86f65cae5ee7b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java @@ -32,6 +32,7 @@ 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.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; @@ -58,8 +59,30 @@ public CleanPlanActionExecutor(HoodieEngineContext context, this.extraMetadata = extraMetadata; } - protected Option createCleanerPlan() { - return execute(); + private int getCommitsSinceLastCleaning() { + Option lastCleanInstant = table.getActiveTimeline().getCleanerTimeline().filterCompletedInstants().lastInstant(); + HoodieTimeline commitTimeline = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + + String latestCleanTs; + int numCommits = 0; + if (lastCleanInstant.isPresent()) { + latestCleanTs = lastCleanInstant.get().getTimestamp(); + numCommits = commitTimeline.findInstantsAfter(latestCleanTs).countInstants(); + } else { + numCommits = commitTimeline.countInstants(); + } + + return numCommits; + } + + private boolean needsCleaning(CleaningTriggerStrategy strategy) { + if (strategy == CleaningTriggerStrategy.NUM_COMMITS) { + int numberOfCommits = getCommitsSinceLastCleaning(); + int maxInlineCommitsForNextClean = config.getCleaningMaxCommits(); + return numberOfCommits >= maxInlineCommitsForNextClean; + } else { + throw new HoodieException("Unsupported cleaning trigger strategy: " + config.getCleaningTriggerStrategy()); + } } /** @@ -128,6 +151,9 @@ protected Option requestClean(String startCleanTime) { @Override public Option execute() { + if (!needsCleaning(config.getCleaningTriggerStrategy())) { + return Option.empty(); + } // Plan a new clean action return requestClean(instantTime); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleaningTriggerStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleaningTriggerStrategy.java new file mode 100644 index 0000000000000..f1ffad261694d --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleaningTriggerStrategy.java @@ -0,0 +1,24 @@ +/* + * 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; + +public enum CleaningTriggerStrategy { + // trigger cleaning when reach n commits + NUM_COMMITS +} diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java index d81b76b0f4577..4539c6de3b1ac 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestHoodieConcatHandle.java @@ -117,7 +117,7 @@ public void testInsert() throws Exception { records1.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); int startInstant = 1; - String firstCommitTime = makeNewCommitTime(startInstant++); + String firstCommitTime = makeNewCommitTime(startInstant++, "%09d"); // First insert writeClient.startCommitWithTime(firstCommitTime); writeClient.insert(records1, firstCommitTime); @@ -145,7 +145,7 @@ public void testInsert() throws Exception { records2.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1)); records2.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2)); - String newCommitTime = makeNewCommitTime(startInstant++); + String newCommitTime = makeNewCommitTime(startInstant++, "%09d"); writeClient.startCommitWithTime(newCommitTime); // Second insert is the same as the _row_key of the first one,test allowDuplicateInserts writeClient.insert(records2, newCommitTime); @@ -183,7 +183,7 @@ public void testInsertWithDataGenerator(boolean mergeAllowDuplicateOnInsertsEnab HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{partitionPath}); int startInstant = 1; - String firstCommitTime = makeNewCommitTime(startInstant++); + String firstCommitTime = makeNewCommitTime(startInstant++, "%09d"); List records1 = dataGenerator.generateInserts(firstCommitTime, 100); // First insert @@ -200,7 +200,7 @@ public void testInsertWithDataGenerator(boolean mergeAllowDuplicateOnInsertsEnab assertTrue(filter.mightContain(record.getRecordKey())); } - String newCommitTime = makeNewCommitTime(startInstant++); + String newCommitTime = makeNewCommitTime(startInstant++, "%09d"); List records2 = dataGenerator.generateUpdates(newCommitTime, 100); writeClient.startCommitWithTime(newCommitTime); // Second insert is the same as the _row_key of the first one,test allowDuplicateInserts diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java index 793b26703011e..8f296d5106174 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java @@ -123,7 +123,7 @@ public void testUpdateRecords() throws Exception { // Prepare the AvroParquetIO HoodieWriteConfig config = makeHoodieClientConfig(); int startInstant = 1; - String firstCommitTime = makeNewCommitTime(startInstant++); + String firstCommitTime = makeNewCommitTime(startInstant++, "%09d"); HoodieJavaWriteClient writeClient = getHoodieWriteClient(config); writeClient.startCommitWithTime(firstCommitTime); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -185,7 +185,7 @@ public void testUpdateRecords() throws Exception { List updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1); - String newCommitTime = makeNewCommitTime(startInstant++); + String newCommitTime = makeNewCommitTime(startInstant++, "%09d"); metaClient = HoodieTableMetaClient.reload(metaClient); writeClient.startCommitWithTime(newCommitTime); List statuses = writeClient.upsert(updatedRecords, newCommitTime); 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 552e85af4c66b..56cfe959bbabf 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 @@ -26,7 +26,6 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieClusteringStrategy; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSliceInfo; @@ -88,14 +87,11 @@ import org.apache.spark.api.java.JavaRDD; 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 org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; @@ -523,7 +519,8 @@ private void testInsertAndCleanByCommits( HoodieCleaningPolicy.KEEP_LATEST_COMMITS); // Keep doing some writes and clean inline. Make sure we have expected number of files remaining. - makeIncrementalCommitTimes(8).forEach(newCommitTime -> { + for (int i = 0; i < 8; i++) { + String newCommitTime = makeNewCommitTime(); try { client.startCommitWithTime(newCommitTime); List records = recordUpsertGenWrappedFunction.apply(newCommitTime, 100); @@ -563,7 +560,7 @@ private void testInsertAndCleanByCommits( } catch (IOException ioe) { throw new RuntimeException(ioe); } - }); + } } /** @@ -629,16 +626,20 @@ private void testFailedInsertAndCleanByCommits( * * @param config HoodieWriteConfig */ - private List runCleaner(HoodieWriteConfig config) throws IOException { - return runCleaner(config, false, 1); + protected List runCleaner(HoodieWriteConfig config) throws IOException { + return runCleaner(config, false, 1, false); } - private List runCleaner(HoodieWriteConfig config, int firstCommitSequence) throws IOException { - return runCleaner(config, false, firstCommitSequence); + protected List runCleanerWithInstantFormat(HoodieWriteConfig config, boolean needInstantInHudiFormat) throws IOException { + return runCleaner(config, false, 1, needInstantInHudiFormat); + } + + protected List runCleaner(HoodieWriteConfig config, int firstCommitSequence, boolean needInstantInHudiFormat) throws IOException { + return runCleaner(config, false, firstCommitSequence, needInstantInHudiFormat); } protected List runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure) throws IOException { - return runCleaner(config, simulateRetryFailure, 1); + return runCleaner(config, simulateRetryFailure, 1, false); } /** @@ -646,9 +647,9 @@ protected List runCleaner(HoodieWriteConfig config, boolean sim * * @param config HoodieWriteConfig */ - private List runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure, int firstCommitSequence) throws IOException { + protected List runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure, Integer firstCommitSequence, boolean needInstantInHudiFormat) throws IOException { SparkRDDWriteClient writeClient = getHoodieWriteClient(config); - String cleanInstantTs = makeNewCommitTime(firstCommitSequence); + String cleanInstantTs = needInstantInHudiFormat ? makeNewCommitTime(firstCommitSequence, "%014d") : makeNewCommitTime(firstCommitSequence, "%09d"); HoodieCleanMetadata cleanMetadata1 = writeClient.clean(cleanInstantTs); if (null == cleanMetadata1) { @@ -706,116 +707,6 @@ private List runCleaner(HoodieWriteConfig config, boolean simul return new ArrayList<>(cleanStatMap.values()); } - /** - * Test Hudi COW Table Cleaner - Keep the latest file versions policy. - */ - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception { - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) - .build(); - - HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); - HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); - - final String p0 = "2020/01/01"; - final String p1 = "2020/01/02"; - final Map> bootstrapMapping = enableBootstrapSourceClean - ? generateBootstrapIndexAndSourceData(p0, p1) : null; - - // make 1 commit, with 1 file per partition - final String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() - : UUID.randomUUID().toString(); - final String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() - : UUID.randomUUID().toString(); - - Map>> c1PartitionToFilesNameLengthMap = new HashMap<>(); - c1PartitionToFilesNameLengthMap.put(p0, Collections.singletonList(Pair.of(file1P0C0, 100))); - c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, 200))); - testTable.doWriteOperation("00000000000001", WriteOperationType.INSERT, Arrays.asList(p0, p1), - c1PartitionToFilesNameLengthMap, false, false); - - List hoodieCleanStatsOne = runCleaner(config); - assertEquals(0, hoodieCleanStatsOne.size(), "Must not clean any files"); - assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); - assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); - - // make next commit, with 1 insert & 1 update per partition - final String file2P0C1 = UUID.randomUUID().toString(); - final String file2P1C1 = UUID.randomUUID().toString(); - Map>> c2PartitionToFilesNameLengthMap = new HashMap<>(); - c2PartitionToFilesNameLengthMap.put(p0, Arrays.asList(Pair.of(file1P0C0, 101), Pair.of(file2P0C1, 100))); - c2PartitionToFilesNameLengthMap.put(p1, Arrays.asList(Pair.of(file1P1C0, 201), Pair.of(file2P1C1, 200))); - testTable.doWriteOperation("00000000000002", WriteOperationType.UPSERT, Collections.emptyList(), - c2PartitionToFilesNameLengthMap, false, false); - - // enableBootstrapSourceClean would delete the bootstrap base file at the same time - List hoodieCleanStatsTwo = runCleaner(config, 1); - HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsTwo, p0); - assertEquals(enableBootstrapSourceClean ? 2 : 1, cleanStat.getSuccessDeleteFiles().size() - + (cleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 - : cleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least 1 file"); - - if (enableBootstrapSourceClean) { - HoodieFileStatus fstatus = - bootstrapMapping.get(p0).get(0).getBootstrapFileStatus(); - // This ensures full path is recorded in metadata. - assertTrue(cleanStat.getSuccessDeleteBootstrapBaseFiles().contains(fstatus.getPath().getUri()), - "Successful delete files were " + cleanStat.getSuccessDeleteBootstrapBaseFiles() - + " but did not contain " + fstatus.getPath().getUri()); - assertFalse(Files.exists(Paths.get(bootstrapMapping.get( - p0).get(0).getBootstrapFileStatus().getPath().getUri()))); - } - - cleanStat = getCleanStat(hoodieCleanStatsTwo, p1); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); - assertTrue(testTable.baseFileExists(p1, "00000000000002", file2P1C1)); - assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); - assertFalse(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); - assertEquals(enableBootstrapSourceClean ? 2 : 1, cleanStat.getSuccessDeleteFiles().size() - + (cleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 - : cleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least 1 file"); - - if (enableBootstrapSourceClean) { - HoodieFileStatus fstatus = - bootstrapMapping.get(p1).get(0).getBootstrapFileStatus(); - // This ensures full path is recorded in metadata. - assertTrue(cleanStat.getSuccessDeleteBootstrapBaseFiles().contains(fstatus.getPath().getUri()), - "Successful delete files were " + cleanStat.getSuccessDeleteBootstrapBaseFiles() - + " but did not contain " + fstatus.getPath().getUri()); - assertFalse(Files.exists(Paths.get(bootstrapMapping.get( - p1).get(0).getBootstrapFileStatus().getPath().getUri()))); - } - - // make next commit, with 2 updates to existing files, and 1 insert - final String file3P0C2 = UUID.randomUUID().toString(); - Map>> c3PartitionToFilesNameLengthMap = new HashMap<>(); - c3PartitionToFilesNameLengthMap.put(p0, Arrays.asList(Pair.of(file1P0C0, 102), Pair.of(file2P0C1, 101), - Pair.of(file3P0C2, 100))); - testTable.doWriteOperation("00000000000003", WriteOperationType.UPSERT, Collections.emptyList(), - c3PartitionToFilesNameLengthMap, false, false); - - List hoodieCleanStatsThree = runCleaner(config, 3); - assertEquals(2, - getCleanStat(hoodieCleanStatsThree, p0) - .getSuccessDeleteFiles().size(), "Must clean two files"); - assertFalse(testTable.baseFileExists(p0, "00000000000002", file1P0C0)); - assertFalse(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); - assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); - - // No cleaning on partially written file, with no commit. - testTable.forCommit("00000000000004").withBaseFilesInPartition(p0, file3P0C2); - - List hoodieCleanStatsFour = runCleaner(config); - assertEquals(0, hoodieCleanStatsFour.size(), "Must not clean any files"); - assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); - } - @Test public void testCleanEmptyInstants() throws Exception { HoodieWriteConfig config = @@ -831,16 +722,19 @@ public void testCleanEmptyInstants() throws Exception { int cleanCount = 20; int startInstant = 1; - for (int i = 0; i < commitCount; i++, startInstant++) { - String commitTime = makeNewCommitTime(startInstant); - HoodieTestTable.of(metaClient).addCommit(commitTime); - } for (int i = 0; i < cleanCount; i++, startInstant++) { - String commitTime = makeNewCommitTime(startInstant); + String commitTime = makeNewCommitTime(startInstant, "%09d"); createCleanMetadata(commitTime + "", false, true); } + int instantClean = startInstant; + + for (int i = 0; i < commitCount; i++, startInstant++) { + String commitTime = makeNewCommitTime(startInstant, "%09d"); + HoodieTestTable.of(metaClient).addCommit(commitTime); + } + List cleanStats = runCleaner(config); HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); @@ -852,7 +746,7 @@ public void testCleanEmptyInstants() throws Exception { assertEquals(--cleanCount, timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--startInstant))); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); cleanStats = runCleaner(config); timeline = metaClient.reloadActiveTimeline(); @@ -865,91 +759,7 @@ public void testCleanEmptyInstants() throws Exception { assertEquals(--cleanCount, timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants().countInstants()); assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--startInstant))); - } - - - /** - * Test HoodieTable.clean() Cleaning by versions logic for MOR table with Log files. - */ - @Test - public void testKeepLatestFileVersionsMOR() throws Exception { - - HoodieWriteConfig config = - HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(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); - - List hoodieCleanStats = runCleaner(config); - assertEquals(3, - getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() - .size(), "Must clean three files, one base 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)); - } - - /** - * 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 base 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)); + CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION)).filterInflightsAndRequested().containsInstant(makeNewCommitTime(--instantClean, "%09d"))); } @Test @@ -987,7 +797,7 @@ public void testCleanWithReplaceCommits() throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); - List hoodieCleanStatsOne = runCleaner(config); + List hoodieCleanStatsOne = runCleanerWithInstantFormat(config, true); 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)); @@ -1001,7 +811,7 @@ public void testCleanWithReplaceCommits() throws Exception { testTable.addReplaceCommit("00000000000002", Option.of(replaceMetadata.getKey()), Option.empty(), replaceMetadata.getValue()); // run cleaner - List hoodieCleanStatsTwo = runCleaner(config); + List hoodieCleanStatsTwo = runCleanerWithInstantFormat(config, true); 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)); @@ -1015,7 +825,7 @@ public void testCleanWithReplaceCommits() throws Exception { testTable.addReplaceCommit("00000000000003", Option.of(replaceMetadata.getKey()), Option.empty(), replaceMetadata.getValue()); // run cleaner - List hoodieCleanStatsThree = runCleaner(config); + List hoodieCleanStatsThree = runCleanerWithInstantFormat(config, true); 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)); @@ -1030,7 +840,7 @@ public void testCleanWithReplaceCommits() throws Exception { testTable.addReplaceCommit("00000000000004", Option.of(replaceMetadata.getKey()), Option.empty(), replaceMetadata.getValue()); // run cleaner - List hoodieCleanStatsFour = runCleaner(config, 5); + List hoodieCleanStatsFour = runCleaner(config, 5, true); assertTrue(testTable.baseFileExists(p0, "00000000000004", file4P0C3)); assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); assertTrue(testTable.baseFileExists(p1, "00000000000003", file3P1C2)); @@ -1045,7 +855,7 @@ public void testCleanWithReplaceCommits() throws Exception { replaceMetadata = generateReplaceCommitMetadata("00000000000006", p0, file3P1C2, file4P1C4); testTable.addReplaceCommit("00000000000006", Option.of(replaceMetadata.getKey()), Option.empty(), replaceMetadata.getValue()); - List hoodieCleanStatsFive = runCleaner(config, 7); + List hoodieCleanStatsFive = runCleaner(config, 7, true); assertTrue(testTable.baseFileExists(p0, "00000000000004", file4P0C3)); assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); assertTrue(testTable.baseFileExists(p1, "00000000000003", file3P1C2)); @@ -1252,147 +1062,6 @@ private static void assertCleanMetadataPathEquals(Map expected, } } - protected static Stream argumentsForTestKeepLatestCommits() { - return Stream.of( - Arguments.of(false, false, false), - Arguments.of(true, false, false), - Arguments.of(false, true, false), - Arguments.of(false, false, true) - ); - } - - /** - * Test HoodieTable.clean() Cleaning by commit logic for COW table. - */ - @ParameterizedTest - @MethodSource("argumentsForTestKeepLatestCommits") - public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withIncrementalCleaningMode(enableIncrementalClean) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) - .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) - .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build()) - .build(); - - HoodieTestTable testTable = HoodieTestTable.of(metaClient); - String p0 = "2020/01/01"; - String p1 = "2020/01/02"; - Map> bootstrapMapping = enableBootstrapSourceClean ? generateBootstrapIndexAndSourceData(p0, p1) : null; - - // make 1 commit, with 1 file per partition - String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() - : UUID.randomUUID().toString(); - String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() - : UUID.randomUUID().toString(); - testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - - HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000001", - 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, simulateFailureRetry); - 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 commit, with 1 insert & 1 update per partition - Map partitionAndFileId002 = testTable.addInflightCommit("00000000000002").getFileIdsWithBaseFilesInPartitions(p0, p1); - String file2P0C1 = partitionAndFileId002.get(p0); - String file2P1C1 = partitionAndFileId002.get(p1); - testTable.forCommit("00000000000002").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); - commitMetadata = generateCommitMetadata("00000000000002", new HashMap>() { - { - put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1)); - put(p1, CollectionUtils.createImmutableList(file1P1C0, file2P1C1)); - } - }); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000002"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - List hoodieCleanStatsTwo = runCleaner(config, simulateFailureRetry); - assertEquals(0, hoodieCleanStatsTwo.size(), "Must not scan any partitions and clean any files"); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); - assertTrue(testTable.baseFileExists(p1, "00000000000002", file2P1C1)); - assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); - assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); - - // make next commit, with 2 updates to existing files, and 1 insert - String file3P0C2 = testTable.addInflightCommit("00000000000003") - .withBaseFilesInPartition(p0, file1P0C0) - .withBaseFilesInPartition(p0, file2P0C1) - .getFileIdsWithBaseFilesInPartitions(p0).get(p0); - commitMetadata = generateCommitMetadata("00000000000003", - CollectionUtils.createImmutableMap( - p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2))); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000003"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - - List hoodieCleanStatsThree = runCleaner(config, simulateFailureRetry); - assertEquals(0, hoodieCleanStatsThree.size(), - "Must not clean any file. We have to keep 1 version before the latest commit time to keep"); - assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); - - // make next commit, with 2 updates to existing files, and 1 insert - String file4P0C3 = testTable.addInflightCommit("00000000000004") - .withBaseFilesInPartition(p0, file1P0C0) - .withBaseFilesInPartition(p0, file2P0C1) - .getFileIdsWithBaseFilesInPartitions(p0).get(p0); - commitMetadata = generateCommitMetadata("00000000000004", - CollectionUtils.createImmutableMap( - p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3))); - metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000004"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - - List hoodieCleanStatsFour = runCleaner(config, simulateFailureRetry); - // enableBootstrapSourceClean would delete the bootstrap base file as the same time - HoodieCleanStat partitionCleanStat = getCleanStat(hoodieCleanStatsFour, p0); - - assertEquals(enableBootstrapSourceClean ? 2 : 1, partitionCleanStat.getSuccessDeleteFiles().size() - + (partitionCleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 - : partitionCleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least one old file"); - assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file1P0C0)); - assertTrue(testTable.baseFileExists(p0, "00000000000003", file1P0C0)); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); - assertTrue(testTable.baseFileExists(p0, "00000000000003", file2P0C1)); - assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); - assertTrue(testTable.baseFileExists(p0, "00000000000004", file4P0C3)); - if (enableBootstrapSourceClean) { - assertFalse(Files.exists(Paths.get(bootstrapMapping.get( - p0).get(0).getBootstrapFileStatus().getPath().getUri()))); - } - - // No cleaning on partially written file, with no commit. - testTable.forCommit("00000000000005").withBaseFilesInPartition(p0, file3P0C2); - commitMetadata = generateCommitMetadata("00000000000005", - CollectionUtils.createImmutableMap(p0, CollectionUtils.createImmutableList(file3P0C2))); - metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000005")); - metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000005"), - Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - List hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry); - HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsFive, p0); - assertNull(cleanStat, "Must not clean any files"); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file1P0C0)); - assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); - assertTrue(testTable.baseFileExists(p0, "00000000000005", file3P0C2)); - } - /** * Generate Bootstrap index, bootstrap base file and corresponding metaClient. * @return Partition to BootstrapFileMapping Map @@ -1528,7 +1197,7 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) .build(); - String commitTime = makeNewCommitTime(1); + String commitTime = makeNewCommitTime(1, "%09d"); List cleanerFileNames = Arrays.asList( HoodieTimeline.makeRequestedCleanerFileName(commitTime), HoodieTimeline.makeInflightCleanerFileName(commitTime)); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java index 961523eb6b993..f44d67e83398b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java @@ -18,42 +18,437 @@ package org.apache.hudi.table.functional; +import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieTableType; +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.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CollectionUtils; 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.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.table.TestCleaner; +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 org.junit.jupiter.params.provider.ValueSource; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; import java.time.Instant; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; -import java.util.Map; import java.util.List; +import java.util.Map; import java.util.UUID; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNull; +/** + * Tests covering different clean plan policies/strategies. + */ public class TestCleanPlanExecutor extends TestCleaner { + @Test + public void testInvalidCleaningTriggerStrategy() { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withIncrementalCleaningMode(true) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withCleanBootstrapBaseFileEnabled(true) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2) + .withCleaningTriggerStrategy("invalid_strategy").build()) + .build(); + Exception e = assertThrows(IllegalArgumentException.class, () -> { + runCleaner(config, true); + }, "should fail when invalid trigger strategy is provided!"); + assertTrue(e.getMessage().contains("No enum constant org.apache.hudi.table.action.clean.CleaningTriggerStrategy.invalid_strategy")); + } + + private static Stream argumentsForTestKeepLatestCommits() { + return Stream.of( + Arguments.of(false, false, false), + Arguments.of(true, false, false), + Arguments.of(false, true, false), + Arguments.of(false, false, true) + ); + } + + /** + * Test HoodieTable.clean() Cleaning by commit logic for COW table. + */ + @ParameterizedTest + @MethodSource("argumentsForTestKeepLatestCommits") + public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withIncrementalCleaningMode(enableIncrementalClean) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) + .retainCommits(2) + .withMaxCommitsBeforeCleaning(2).build()) + .build(); + + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + String p0 = "2020/01/01"; + String p1 = "2020/01/02"; + Map> bootstrapMapping = enableBootstrapSourceClean ? generateBootstrapIndexAndSourceData(p0, p1) : null; + + // make 1 commit, with 1 file per partition + String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() + : UUID.randomUUID().toString(); + String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() + : UUID.randomUUID().toString(); + testTable.addInflightCommit("00000000000001").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); + + HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000001", + Collections.unmodifiableMap(new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0)); + put(p1, CollectionUtils.createImmutableList(file1P1C0)); + } + }) + ); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000001"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + metaClient = HoodieTableMetaClient.reload(metaClient); + + List hoodieCleanStatsOne = runCleaner(config, simulateFailureRetry, 2, true); + 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 commit, with 1 insert & 1 update per partition + Map partitionAndFileId002 = testTable.addInflightCommit("00000000000003").getFileIdsWithBaseFilesInPartitions(p0, p1); + String file2P0C1 = partitionAndFileId002.get(p0); + String file2P1C1 = partitionAndFileId002.get(p1); + testTable.forCommit("00000000000003").withBaseFilesInPartition(p0, file1P0C0).withBaseFilesInPartition(p1, file1P1C0); + commitMetadata = generateCommitMetadata("00000000000003", new HashMap>() { + { + put(p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1)); + put(p1, CollectionUtils.createImmutableList(file1P1C0, file2P1C1)); + } + }); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000003"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + List hoodieCleanStatsTwo = runCleaner(config, simulateFailureRetry, 4, true); + assertEquals(0, hoodieCleanStatsTwo.size(), "Must not scan any partitions and clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file2P0C1)); + assertTrue(testTable.baseFileExists(p1, "00000000000003", file2P1C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next commit, with 2 updates to existing files, and 1 insert + String file3P0C2 = testTable.addInflightCommit("00000000000005") + .withBaseFilesInPartition(p0, file1P0C0) + .withBaseFilesInPartition(p0, file2P0C1) + .getFileIdsWithBaseFilesInPartitions(p0).get(p0); + commitMetadata = generateCommitMetadata("00000000000003", + CollectionUtils.createImmutableMap( + p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2))); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000005"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + List hoodieCleanStatsThree = runCleaner(config, simulateFailureRetry, 6, true); + assertEquals(0, hoodieCleanStatsThree.size(), + "Must not clean any file. We have to keep 1 version before the latest commit time to keep"); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + + // make next commit, with 2 updates to existing files, and 1 insert + String file4P0C3 = testTable.addInflightCommit("00000000000007") + .withBaseFilesInPartition(p0, file1P0C0) + .withBaseFilesInPartition(p0, file2P0C1) + .getFileIdsWithBaseFilesInPartitions(p0).get(p0); + commitMetadata = generateCommitMetadata("00000000000004", + CollectionUtils.createImmutableMap( + p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3))); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000007"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + List hoodieCleanStatsFour = runCleaner(config, simulateFailureRetry, 8, true); + // enableBootstrapSourceClean would delete the bootstrap base file as the same time + HoodieCleanStat partitionCleanStat = getCleanStat(hoodieCleanStatsFour, p0); + + assertEquals(enableBootstrapSourceClean ? 2 : 1, partitionCleanStat.getSuccessDeleteFiles().size() + + (partitionCleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 + : partitionCleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least one old file"); + assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file1P0C0)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file1P0C0)); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file3P0C2)); + assertTrue(testTable.baseFileExists(p0, "00000000000007", file4P0C3)); + if (enableBootstrapSourceClean) { + assertFalse(Files.exists(Paths.get(bootstrapMapping.get( + p0).get(0).getBootstrapFileStatus().getPath().getUri()))); + } + + metaClient = HoodieTableMetaClient.reload(metaClient); + + String file5P0C4 = testTable.addInflightCommit("00000000000009") + .withBaseFilesInPartition(p0, file1P0C0) + .withBaseFilesInPartition(p0, file2P0C1) + .getFileIdsWithBaseFilesInPartitions(p0).get(p0); + commitMetadata = generateCommitMetadata("00000000000009", CollectionUtils.createImmutableMap( + p0, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file5P0C4))); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "00000000000009"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + List hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry, 10, true); + + assertEquals(0, hoodieCleanStatsFive.size(), "Must not clean any files since at least 2 commits are needed from last clean operation before " + + "clean can be scheduled again"); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file1P0C0)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file1P0C0)); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file3P0C2)); + assertTrue(testTable.baseFileExists(p0, "00000000000007", file4P0C3)); + + // No cleaning on partially written file, with no commit. + testTable.forCommit("00000000000011").withBaseFilesInPartition(p0, file3P0C2); + commitMetadata = generateCommitMetadata("00000000000011", CollectionUtils.createImmutableMap(p0, + CollectionUtils.createImmutableList(file3P0C2))); + metaClient.getActiveTimeline().createNewInstant( + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011")); + metaClient.getActiveTimeline().transitionRequestedToInflight( + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011"), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + List hoodieCleanStatsFive2 = runCleaner(config, simulateFailureRetry, 12, true); + HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsFive2, p0); + assertNull(cleanStat, "Must not clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000005", file3P0C2)); + assertTrue(testTable.baseFileExists(p0, "00000000000007", file4P0C3)); + } + + /** + * Test Hudi COW Table Cleaner - Keep the latest file versions policy. + */ + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception { + HoodieWriteConfig config = + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean) + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build()) + .build(); + + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + + final String p0 = "2020/01/01"; + final String p1 = "2020/01/02"; + final Map> bootstrapMapping = enableBootstrapSourceClean + ? generateBootstrapIndexAndSourceData(p0, p1) : null; + + // make 1 commit, with 1 file per partition + final String file1P0C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p0).get(0).getFileId() + : UUID.randomUUID().toString(); + final String file1P1C0 = enableBootstrapSourceClean ? bootstrapMapping.get(p1).get(0).getFileId() + : UUID.randomUUID().toString(); + + Map>> c1PartitionToFilesNameLengthMap = new HashMap<>(); + c1PartitionToFilesNameLengthMap.put(p0, Collections.singletonList(Pair.of(file1P0C0, 100))); + c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, 200))); + testTable.doWriteOperation("00000000000001", WriteOperationType.INSERT, Arrays.asList(p0, p1), + c1PartitionToFilesNameLengthMap, false, false); + + List hoodieCleanStatsOne = runCleanerWithInstantFormat(config, true); + assertEquals(0, hoodieCleanStatsOne.size(), "Must not clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + + // make next commit, with 1 insert & 1 update per partition + final String file2P0C1 = UUID.randomUUID().toString(); + final String file2P1C1 = UUID.randomUUID().toString(); + Map>> c2PartitionToFilesNameLengthMap = new HashMap<>(); + c2PartitionToFilesNameLengthMap.put(p0, Arrays.asList(Pair.of(file1P0C0, 101), Pair.of(file2P0C1, 100))); + c2PartitionToFilesNameLengthMap.put(p1, Arrays.asList(Pair.of(file1P1C0, 201), Pair.of(file2P1C1, 200))); + testTable.doWriteOperation("00000000000002", WriteOperationType.UPSERT, Collections.emptyList(), + c2PartitionToFilesNameLengthMap, false, false); + + // enableBootstrapSourceClean would delete the bootstrap base file at the same time + List hoodieCleanStatsTwo = runCleaner(config, 1, true); + HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsTwo, p0); + assertEquals(enableBootstrapSourceClean ? 2 : 1, cleanStat.getSuccessDeleteFiles().size() + + (cleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 + : cleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least 1 file"); + + if (enableBootstrapSourceClean) { + HoodieFileStatus fstatus = + bootstrapMapping.get(p0).get(0).getBootstrapFileStatus(); + // This ensures full path is recorded in metadata. + assertTrue(cleanStat.getSuccessDeleteBootstrapBaseFiles().contains(fstatus.getPath().getUri()), + "Successful delete files were " + cleanStat.getSuccessDeleteBootstrapBaseFiles() + + " but did not contain " + fstatus.getPath().getUri()); + assertFalse(Files.exists(Paths.get(bootstrapMapping.get( + p0).get(0).getBootstrapFileStatus().getPath().getUri()))); + } + + cleanStat = getCleanStat(hoodieCleanStatsTwo, p1); + assertTrue(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p1, "00000000000002", file2P1C1)); + assertFalse(testTable.baseFileExists(p0, "00000000000001", file1P0C0)); + assertFalse(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); + assertEquals(enableBootstrapSourceClean ? 2 : 1, cleanStat.getSuccessDeleteFiles().size() + + (cleanStat.getSuccessDeleteBootstrapBaseFiles() == null ? 0 + : cleanStat.getSuccessDeleteBootstrapBaseFiles().size()), "Must clean at least 1 file"); + + if (enableBootstrapSourceClean) { + HoodieFileStatus fstatus = + bootstrapMapping.get(p1).get(0).getBootstrapFileStatus(); + // This ensures full path is recorded in metadata. + assertTrue(cleanStat.getSuccessDeleteBootstrapBaseFiles().contains(fstatus.getPath().getUri()), + "Successful delete files were " + cleanStat.getSuccessDeleteBootstrapBaseFiles() + + " but did not contain " + fstatus.getPath().getUri()); + assertFalse(Files.exists(Paths.get(bootstrapMapping.get( + p1).get(0).getBootstrapFileStatus().getPath().getUri()))); + } + + // make next commit, with 2 updates to existing files, and 1 insert + final String file3P0C2 = UUID.randomUUID().toString(); + Map>> c3PartitionToFilesNameLengthMap = new HashMap<>(); + c3PartitionToFilesNameLengthMap.put(p0, Arrays.asList(Pair.of(file1P0C0, 102), Pair.of(file2P0C1, 101), + Pair.of(file3P0C2, 100))); + testTable.doWriteOperation("00000000000003", WriteOperationType.UPSERT, Collections.emptyList(), + c3PartitionToFilesNameLengthMap, false, false); + + List hoodieCleanStatsThree = runCleaner(config, 3, true); + assertEquals(2, + getCleanStat(hoodieCleanStatsThree, p0) + .getSuccessDeleteFiles().size(), "Must clean two files"); + assertFalse(testTable.baseFileExists(p0, "00000000000002", file1P0C0)); + assertFalse(testTable.baseFileExists(p0, "00000000000002", file2P0C1)); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); + + // No cleaning on partially written file, with no commit. + testTable.forCommit("00000000000004").withBaseFilesInPartition(p0, file3P0C2); + + List hoodieCleanStatsFour = runCleaner(config); + assertEquals(0, hoodieCleanStatsFour.size(), "Must not clean any files"); + assertTrue(testTable.baseFileExists(p0, "00000000000003", file3P0C2)); + } + + /** + * Test HoodieTable.clean() Cleaning by versions logic for MOR table with Log files. + */ + @Test + public void testKeepLatestFileVersionsMOR() throws Exception { + + HoodieWriteConfig config = + HoodieWriteConfig.newBuilder().withPath(basePath) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(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); + + List hoodieCleanStats = runCleaner(config); + assertEquals(3, + getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles() + .size(), "Must clean three files, one base 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)); + } + + /** + * 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 base 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)); + } + /** * Tests cleaning service based on number of hours retained. */ 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 6f49c69960fc1..5f9aab84d0e6d 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 @@ -141,8 +141,8 @@ public static HoodieTestTable of(HoodieTableMetaClient metaClient) { return new HoodieTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient); } - public static String makeNewCommitTime(int sequence) { - return String.format("%09d", sequence); + public static String makeNewCommitTime(int sequence, String instantFormat) { + return String.format(instantFormat, sequence); } public static String makeNewCommitTime() { @@ -153,14 +153,6 @@ public static String makeNewCommitTime(Instant dateTime) { return HoodieActiveTimeline.formatDate(Date.from(dateTime)); } - public static List makeIncrementalCommitTimes(int num) { - return makeIncrementalCommitTimes(num, 1); - } - - public static List makeIncrementalCommitTimes(int num, int firstOffsetSeconds) { - return makeIncrementalCommitTimes(num, firstOffsetSeconds, 0); - } - public static List makeIncrementalCommitTimes(int num, int firstOffsetSeconds, int deltaSecs) { final Instant now = Instant.now(); return IntStream.range(0, num) From 9b6e138af27e600410a63f96065baf057300c071 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Mon, 21 Mar 2022 17:35:06 -0700 Subject: [PATCH 32/33] [HUDI-3640] Set SimpleKeyGenerator as default in 2to3 table upgrade for Spark engine (#5075) --- .../upgrade/TwoToThreeUpgradeHandler.java | 10 ++- .../upgrade/TestTwoToThreeUpgradeHandler.java | 26 +++++-- .../JavaSortAndSizeExecutionStrategy.java | 7 +- .../table/upgrade/TestUpgradeDowngrade.java | 70 +++++++++++++++++++ 4 files changed, 104 insertions(+), 9 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java index c13d21ec201a0..8352ada1126e7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToThreeUpgradeHandler.java @@ -20,6 +20,7 @@ package org.apache.hudi.table.upgrade; import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; @@ -34,6 +35,8 @@ * UpgradeHandler to assist in upgrading {@link org.apache.hudi.table.HoodieTable} from version 2 to 3. */ public class TwoToThreeUpgradeHandler implements UpgradeHandler { + public static final String SPARK_SIMPLE_KEY_GENERATOR = "org.apache.hudi.keygen.SimpleKeyGenerator"; + @Override public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, SupportsUpgradeDowngrade upgradeDowngradeHelper) { if (config.isMetadataTableEnabled()) { @@ -47,8 +50,13 @@ public Map upgrade(HoodieWriteConfig config, HoodieEngin tablePropsToAdd.put(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, config.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); String keyGenClassName = Option.ofNullable(config.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) .orElse(config.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME)); + if (keyGenClassName == null && config.getEngineType() == EngineType.SPARK) { + // For Spark, if the key generator class is not configured by user, + // set it to SimpleKeyGenerator as default + keyGenClassName = SPARK_SIMPLE_KEY_GENERATOR; + } ValidationUtils.checkState(keyGenClassName != null, String.format("Missing config: %s or %s", - HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, HoodieWriteConfig.KEYGENERATOR_CLASS_NAME)); + HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, HoodieWriteConfig.KEYGENERATOR_CLASS_NAME)); tablePropsToAdd.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGenClassName); return tablePropsToAdd; } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java index 35928dc7cf319..d6339a9782e1e 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/upgrade/TestTwoToThreeUpgradeHandler.java @@ -21,13 +21,14 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.KeyGenerator; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; import java.util.Map; @@ -58,11 +59,24 @@ void upgradeHandlerShouldRetrieveKeyGeneratorConfig(String keyGenConfigKey) { assertEquals(KeyGenerator.class.getName(), kv.get(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)); } - @Test - void upgradeHandlerShouldThrowWhenKeyGeneratorNotSet() { + @ParameterizedTest + @EnumSource(EngineType.class) + void upgradeHandlerWhenKeyGeneratorNotSet(EngineType engineType) { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withEngineType(engineType) + .forTable("foo") + .withPath("/foo") + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()) + .build(); TwoToThreeUpgradeHandler handler = new TwoToThreeUpgradeHandler(); - Throwable t = assertThrows(IllegalStateException.class, () -> handler - .upgrade(config, null, null, null)); - assertTrue(t.getMessage().startsWith("Missing config:")); + if (engineType == EngineType.SPARK) { + Map kv = handler.upgrade(config, null, null, null); + assertEquals(TwoToThreeUpgradeHandler.SPARK_SIMPLE_KEY_GENERATOR, + kv.get(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)); + } else { + Throwable t = assertThrows(IllegalStateException.class, () -> handler + .upgrade(writeConfig, null, null, null)); + assertTrue(t.getMessage().startsWith("Missing config:")); + } } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java index a33af7ccd0214..7751833fc04b0 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java @@ -19,8 +19,8 @@ package org.apache.hudi.client.clustering.run.strategy; -import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -30,6 +30,8 @@ import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.JavaBulkInsertHelper; + +import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -63,7 +65,8 @@ public List performClusteringWithRecordList( // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files. props.put(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key(), Boolean.FALSE.toString()); props.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes())); - HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build(); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder() + .withEngineType(EngineType.JAVA).withProps(props).build(); return (List) JavaBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); } 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 403b67e554d76..79f20b9f85c75 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 @@ -38,6 +38,8 @@ 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.keygen.SimpleKeyGenerator; +import org.apache.hudi.keygen.TimestampBasedKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.marker.WriteMarkers; @@ -117,6 +119,16 @@ public static Stream downGradeConfigParams() { return Stream.of(data).map(Arguments::of); } + public static Stream twoToThreeUpgradeConfigParams() { + Object[][] data = new Object[][] { + {HoodieTableType.COPY_ON_WRITE, Option.empty()}, + {HoodieTableType.COPY_ON_WRITE, Option.of(TimestampBasedKeyGenerator.class.getName())}, + {HoodieTableType.MERGE_ON_READ, Option.empty()}, + {HoodieTableType.MERGE_ON_READ, Option.of(TimestampBasedKeyGenerator.class.getName())} + }; + return Stream.of(data).map(Arguments::of); + } + @BeforeEach public void setUp() throws Exception { initSparkContexts(); @@ -232,6 +244,51 @@ public void testUpgradeOneToTwo(HoodieTableType tableType) throws IOException { assertTableProps(cfg); } + @ParameterizedTest + @MethodSource("twoToThreeUpgradeConfigParams") + public void testUpgradeTwoToThree( + HoodieTableType tableType, Option keyGeneratorClass) throws IOException { + // init config, table and client. + Map params = new HashMap<>(); + addNewTableParamsToProps(params); + if (tableType == HoodieTableType.MERGE_ON_READ) { + params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name()); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ); + } + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder() + .withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params); + if (keyGeneratorClass.isPresent()) { + cfgBuilder.withKeyGenerator(keyGeneratorClass.get()); + } + HoodieWriteConfig cfg = cfgBuilder.build(); + SparkRDDWriteClient client = getHoodieWriteClient(cfg); + // Write inserts + doInsert(client); + + // downgrade table props + downgradeTableConfigsFromThreeToTwo(cfg); + + // perform upgrade + new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance()) + .run(HoodieTableVersion.THREE, null); + + // verify hoodie.table.version got upgraded + metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build(); + assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.THREE.versionCode()); + assertTableVersionFromPropertyFile(HoodieTableVersion.THREE); + + // verify table props + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + Properties originalProps = cfg.getProps(); + assertEquals(tableConfig.getUrlEncodePartitioning(), + cfg.getStringOrDefault(HoodieTableConfig.URL_ENCODE_PARTITIONING)); + assertEquals(tableConfig.getHiveStylePartitioningEnable(), + cfg.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); + assertEquals(tableConfig.getKeyGeneratorClassName(), originalProps.getOrDefault( + HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName())); + } + @Test public void testUpgradeDowngradeBetweenThreeAndCurrentVersion() throws IOException { // init config, table and client. @@ -298,6 +355,19 @@ private void downgradeTableConfigsFromTwoToOne(HoodieWriteConfig cfg) throws IOE metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ONE); } + private void downgradeTableConfigsFromThreeToTwo(HoodieWriteConfig cfg) throws IOException { + Properties properties = new Properties(cfg.getProps()); + properties.remove(HoodieTableConfig.URL_ENCODE_PARTITIONING.key()); + properties.remove(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key()); + properties.remove(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key()); + properties.remove(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key()); + properties.setProperty(HoodieTableConfig.VERSION.key(), "2"); + + metaClient = HoodieTestUtils.init(hadoopConf, basePath, getTableType(), properties); + // set hoodie.table.version to 2 in hoodie.properties file + metaClient.getTableConfig().setTableVersion(HoodieTableVersion.TWO); + } + private void assertTableProps(HoodieWriteConfig cfg) { HoodieTableConfig tableConfig = metaClient.getTableConfig(); Properties originalProps = cfg.getProps(); From 5f570ea151d0212ab1bb2d1f5693035626b76d31 Mon Sep 17 00:00:00 2001 From: Rajesh Mahindra <76502047+rmahindra123@users.noreply.github.com> Date: Mon, 21 Mar 2022 19:56:31 -0700 Subject: [PATCH 33/33] [HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175) - Refactor hive sync tool / config to use reflection and standardize configs Co-authored-by: sivabalan Co-authored-by: Rajesh Mahindra Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com> --- docker/demo/sparksql-incremental.commands | 34 +- .../apache/hudi/config/HoodieWriteConfig.java | 2 +- .../hudi/common/table/HoodieTableConfig.java | 4 +- .../hudi/common/util/ReflectionUtils.java | 18 + .../testsuite/dag/nodes/HiveQueryNode.java | 29 +- .../testsuite/dag/nodes/HiveSyncNode.java | 19 +- .../helpers/HiveServiceProvider.java | 14 +- .../testsuite/job/TestHoodieTestSuiteJob.java | 9 +- .../hudi/connect/utils/KafkaConnectUtils.java | 31 - .../KafkaConnectTransactionServices.java | 47 +- .../java/org/apache/hudi/DataSourceUtils.java | 8 +- .../org/apache/hudi/DataSourceOptions.scala | 301 +++---- .../apache/hudi/HoodieSparkSqlWriter.scala | 108 +-- .../org/apache/hudi/HoodieWriterUtils.scala | 31 +- ...AlterHoodieTableDropPartitionCommand.scala | 21 +- .../CreateHoodieTableAsSelectCommand.scala | 7 +- .../command/DeleteHoodieTableCommand.scala | 5 +- .../command/MergeIntoHoodieTableCommand.scala | 19 +- .../src/test/java/HoodieJavaApp.java | 23 +- .../src/test/java/HoodieJavaGenerateApp.java | 23 +- .../src/test/java/HoodieJavaStreamingApp.java | 21 +- .../apache/hudi/TestDataSourceOptions.scala | 13 +- .../hudi/TestHoodieSparkSqlWriter.scala | 49 -- .../java/org/apache/hudi/dla/DLASyncTool.java | 12 +- .../java/org/apache/hudi/dla/util/Utils.java | 8 +- .../org/apache/hudi/hive/HiveSyncConfig.java | 241 ++++-- .../org/apache/hudi/hive/HiveSyncTool.java | 85 +- .../replication/GlobalHiveSyncConfig.java | 13 +- .../hive/replication/GlobalHiveSyncTool.java | 4 +- .../apache/hudi/hive/TestHiveSyncTool.java | 772 ++++++++---------- .../hudi/hive/testutils/HiveTestUtil.java | 104 +-- hudi-sync/hudi-sync-common/pom.xml | 59 ++ .../hudi/sync/common/AbstractSyncTool.java | 22 +- .../hudi/sync/common/HoodieSyncConfig.java | 188 +++++ .../sync/common/util/SyncUtilHelpers.java | 72 ++ .../sync/common/util/TestSyncUtilHelpers.java | 124 +++ .../resources/log4j-surefire-quiet.properties | 29 + .../test/resources/log4j-surefire.properties | 30 + .../deltastreamer/BootstrapExecutor.java | 21 +- .../utilities/deltastreamer/DeltaSync.java | 46 +- .../HoodieMultiTableDeltaStreamer.java | 4 +- .../utilities/TestHiveIncrementalPuller.java | 50 +- .../HoodieDeltaStreamerTestBase.java | 22 +- 43 files changed, 1523 insertions(+), 1219 deletions(-) create mode 100644 hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java create mode 100644 hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java create mode 100644 hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java create mode 100644 hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties create mode 100644 hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index da61347ec275b..5ea4729b932f2 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -21,6 +21,8 @@ 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.HiveSyncConfig; +import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hadoop.fs.FileSystem; @@ -43,14 +45,14 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr"). option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts"). option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor"). - option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor"). - option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default"). - option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). - option(DataSourceWriteOptions.HIVE_USER.key(), "hive"). - option(DataSourceWriteOptions.HIVE_PASS.key(), "hive"). - option(DataSourceWriteOptions.HIVE_SYNC_ENABLED.key(), "true"). - option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key(), "datestr"). - option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor"). + option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). + option(HiveSyncConfig.HIVE_USER.key(), "hive"). + option(HiveSyncConfig.HIVE_PASS.key(), "hive"). + option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"). + option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"). + option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor"); @@ -75,14 +77,14 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr"). option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts"). option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs"). - option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor_bs"). - option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default"). - option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). - option(DataSourceWriteOptions.HIVE_USER.key(), "hive"). - option(DataSourceWriteOptions.HIVE_PASS.key(), "hive"). - option(DataSourceWriteOptions.HIVE_SYNC_ENABLED.key(), "true"). - option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key(), "datestr"). - option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor_bs"). + option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). + option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). + option(HiveSyncConfig.HIVE_USER.key(), "hive"). + option(HiveSyncConfig.HIVE_PASS.key(), "hive"). + option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"). + option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"). + option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor_bs"); 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 4847d9c91ed44..776c9066767c9 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 @@ -105,7 +105,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final String DELTASTREAMER_CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; public static final ConfigProperty TBL_NAME = ConfigProperty - .key("hoodie.table.name") + .key(HoodieTableConfig.HOODIE_TABLE_NAME_KEY) .noDefaultValue() .withDocumentation("Table name that will be used for registering with metastores like HMS. Needs to be same across runs."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index bc8a5c443d39f..6b8103db3a663 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -82,6 +82,8 @@ public class HoodieTableConfig extends HoodieConfig { public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties"; public static final String HOODIE_PROPERTIES_FILE_BACKUP = "hoodie.properties.backup"; + public static final String HOODIE_WRITE_TABLE_NAME_KEY = "hoodie.datasource.write.table.name"; + public static final String HOODIE_TABLE_NAME_KEY = "hoodie.table.name"; public static final ConfigProperty DATABASE_NAME = ConfigProperty .key("hoodie.database.name") @@ -90,7 +92,7 @@ public class HoodieTableConfig extends HoodieConfig { + "we can set it to limit the table name under a specific database"); public static final ConfigProperty NAME = ConfigProperty - .key("hoodie.table.name") + .key(HOODIE_TABLE_NAME_KEY) .noDefaultValue() .withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index 15e4aaa96f28a..bc48661c88376 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -92,6 +92,24 @@ public static Object loadClass(String clazz, Class[] constructorArgTypes, Obj } } + /** + * Check if the clazz has the target constructor or not. + * + * When catch {@link HoodieException} from {@link #loadClass}, it's inconvenient to say if the exception was thrown + * due to the instantiation's own logic or missing constructor. + * + * TODO: ReflectionUtils should throw a specific exception to indicate Reflection problem. + */ + public static boolean hasConstructor(String clazz, Class[] constructorArgTypes) { + try { + getClass(clazz).getConstructor(constructorArgTypes); + return true; + } catch (NoSuchMethodException e) { + LOG.warn("Unable to instantiate class " + clazz, e); + return false; + } + } + /** * Creates an instance of the given class. Constructor arg types are inferred. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java index bdde58adb19e6..4736133f2c20c 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java @@ -18,17 +18,19 @@ package org.apache.hudi.integ.testsuite.dag.nodes; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.dag.ExecutionContext; import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider; +import org.apache.hudi.sync.common.HoodieSyncConfig; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; /** * A hive query node in the DAG of operations for a workflow. used to perform a hive query with given config. @@ -46,13 +48,14 @@ public HiveQueryNode(DeltaConfig.Config config) { public void execute(ExecutionContext executionContext, int curItrCount) throws Exception { log.info("Executing hive query node {}", this.getName()); this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration()); - HiveSyncConfig hiveSyncConfig = DataSourceUtils - .buildHiveSyncConfig(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() - .getDeltaSyncService().getDeltaSync().getProps(), - executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() - .getDeltaSyncService().getDeltaSync().getCfg().targetBasePath, - executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() - .getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat); + TypedProperties properties = new TypedProperties(); + properties.putAll(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getProps()); + properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getCfg().targetBasePath); + properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat); + HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(properties); this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter()); Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser, hiveSyncConfig.hivePass); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java index 97a1bee07c152..3f0c90f03067c 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java @@ -18,31 +18,30 @@ package org.apache.hudi.integ.testsuite.dag.nodes; -import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.sync.common.util.SyncUtilHelpers; + +import org.apache.hadoop.fs.Path; /** * Represents a hive sync node in the DAG of operations for a workflow. Helps to sync hoodie data to hive table. */ public class HiveSyncNode extends DagNode { - private HiveServiceProvider hiveServiceProvider; public HiveSyncNode(Config config) { this.config = config; - this.hiveServiceProvider = new HiveServiceProvider(config); } @Override public void execute(ExecutionContext executionContext, int curItrCount) throws Exception { log.info("Executing hive sync node"); - this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration()); - this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter()); - this.hiveServiceProvider.stopLocalHiveServiceIfNeeded(); - } - - public HiveServiceProvider getHiveServiceProvider() { - return hiveServiceProvider; + SyncUtilHelpers.runHoodieMetaSync(HiveSyncTool.class.getName(), new TypedProperties(executionContext.getHoodieTestSuiteWriter().getProps()), + executionContext.getHoodieTestSuiteWriter().getConfiguration(), + new Path(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath).getFileSystem(executionContext.getHoodieTestSuiteWriter().getConfiguration()), + executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath, executionContext.getHoodieTestSuiteWriter().getCfg().baseFileFormat); } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java index 85a292c2a2701..a10e4d31fa736 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java @@ -21,6 +21,9 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hive.service.server.HiveServer2; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.testutils.HiveTestService; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; @@ -46,12 +49,17 @@ public void startLocalHiveServiceIfNeeded(Configuration configuration) throws IO } public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) { + HiveSyncTool hiveSyncTool; if (this.config.isHiveLocal()) { - writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync() - .syncHive(getLocalHiveServer().getHiveConf()); + hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), + getLocalHiveServer().getHiveConf(), + FSUtils.getFs(writer.getWriteConfig().getBasePath(), getLocalHiveServer().getHiveConf())); } else { - writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive(); + hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), + getLocalHiveServer().getHiveConf(), + FSUtils.getFs(writer.getWriteConfig().getBasePath(), writer.getConfiguration())); } + hiveSyncTool.syncHoodieTable(); } public void stopLocalHiveServiceIfNeeded() throws IOException { diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index 1b0cef58c630e..62c1d851e2108 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; @@ -173,10 +174,10 @@ private static TypedProperties getProperties() { // Make path selection test suite specific props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); // Hive Configs - props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1"); - props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "table1"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr"); + props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1"); + props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "table1"); + props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName()); props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java index 6a3843075bdaf..cc37de2f2926a 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java @@ -32,8 +32,6 @@ import org.apache.hudi.connect.ControlMessage; import org.apache.hudi.connect.writers.KafkaConnectConfigs; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.CustomAvroKeyGenerator; import org.apache.hudi.keygen.CustomKeyGenerator; @@ -59,7 +57,6 @@ import java.security.NoSuchAlgorithmException; import java.util.Arrays; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -269,32 +266,4 @@ public static List getWriteStatuses(ControlMessage.ParticipantInfo ControlMessage.ConnectWriteStatus connectWriteStatus = participantInfo.getWriteStatus(); return SerializationUtils.deserialize(connectWriteStatus.getSerializedWriteStatus().toByteArray()); } - - /** - * Build Hive Sync Config - * Note: This method is a temporary solution. - * Future solutions can be referred to: https://issues.apache.org/jira/browse/HUDI-3199 - */ - public static HiveSyncConfig buildSyncConfig(TypedProperties props, String tableBasePath) { - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.basePath = tableBasePath; - hiveSyncConfig.usePreApacheInputFormat = props.getBoolean(KafkaConnectConfigs.HIVE_USE_PRE_APACHE_INPUT_FORMAT, false); - hiveSyncConfig.databaseName = props.getString(KafkaConnectConfigs.HIVE_DATABASE, "default"); - hiveSyncConfig.tableName = props.getString(KafkaConnectConfigs.HIVE_TABLE, ""); - hiveSyncConfig.hiveUser = props.getString(KafkaConnectConfigs.HIVE_USER, ""); - hiveSyncConfig.hivePass = props.getString(KafkaConnectConfigs.HIVE_PASS, ""); - hiveSyncConfig.jdbcUrl = props.getString(KafkaConnectConfigs.HIVE_URL, ""); - hiveSyncConfig.partitionFields = props.getStringList(KafkaConnectConfigs.HIVE_PARTITION_FIELDS, ",", Collections.emptyList()); - hiveSyncConfig.partitionValueExtractorClass = - props.getString(KafkaConnectConfigs.HIVE_PARTITION_EXTRACTOR_CLASS, SlashEncodedDayPartitionValueExtractor.class.getName()); - hiveSyncConfig.useJdbc = props.getBoolean(KafkaConnectConfigs.HIVE_USE_JDBC, true); - if (props.containsKey(KafkaConnectConfigs.HIVE_SYNC_MODE)) { - hiveSyncConfig.syncMode = props.getString(KafkaConnectConfigs.HIVE_SYNC_MODE); - } - hiveSyncConfig.autoCreateDatabase = props.getBoolean(KafkaConnectConfigs.HIVE_AUTO_CREATE_DATABASE, true); - hiveSyncConfig.ignoreExceptions = props.getBoolean(KafkaConnectConfigs.HIVE_IGNORE_EXCEPTIONS, false); - hiveSyncConfig.skipROSuffix = props.getBoolean(KafkaConnectConfigs.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE, false); - hiveSyncConfig.supportTimestamp = props.getBoolean(KafkaConnectConfigs.HIVE_SUPPORT_TIMESTAMP_TYPE, false); - return hiveSyncConfig; - } } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java index dae19cc413f3f..65a1d8ae2ddba 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java @@ -30,22 +30,17 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.connect.transaction.TransactionCoordinator; import org.apache.hudi.connect.utils.KafkaConnectUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.HiveSyncTool; -import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncConfig; +import org.apache.hudi.sync.common.util.SyncUtilHelpers; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -54,7 +49,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; /** @@ -167,43 +161,10 @@ private void syncMeta() { if (connectConfigs.isMetaSyncEnabled()) { Set syncClientToolClasses = new HashSet<>( Arrays.asList(connectConfigs.getMetaSyncClasses().split(","))); + FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration()); for (String impl : syncClientToolClasses) { - impl = impl.trim(); - switch (impl) { - case "org.apache.hudi.hive.HiveSyncTool": - syncHive(); - break; - default: - FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration()); - Properties properties = new Properties(); - properties.putAll(connectConfigs.getProps()); - properties.put("basePath", tableBasePath); - AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[] {Properties.class, FileSystem.class}, properties, fs); - syncTool.syncHoodieTable(); - } + SyncUtilHelpers.runHoodieMetaSync(impl.trim(), connectConfigs.getProps(), hadoopConf, fs, tableBasePath, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue()); } } } - - private void syncHive() { - HiveSyncConfig hiveSyncConfig = KafkaConnectUtils.buildSyncConfig(new TypedProperties(connectConfigs.getProps()), tableBasePath); - String url; - if (!StringUtils.isNullOrEmpty(hiveSyncConfig.syncMode) && HiveSyncMode.of(hiveSyncConfig.syncMode) == HiveSyncMode.HMS) { - url = hadoopConf.get(KafkaConnectConfigs.HIVE_METASTORE_URIS); - } else { - url = hiveSyncConfig.jdbcUrl; - } - - LOG.info("Syncing target hoodie table with hive table(" - + hiveSyncConfig.tableName - + "). Hive URL :" - + url - + ", basePath :" + tableBasePath); - LOG.info("Hive Sync Conf => " + hiveSyncConfig); - FileSystem fs = FSUtils.getFs(tableBasePath, hadoopConf); - HiveConf hiveConf = new HiveConf(); - hiveConf.addResource(fs.getConf()); - LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString()); - new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable(); - } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 634389b1e68b1..4042f431d7d56 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -42,6 +42,7 @@ import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; +import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.util.DataTypeUtils; @@ -270,6 +271,11 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRD return dropDuplicates(jssc, incomingHoodieRecords, writeConfig); } + /** + * @deprecated Use {@link HiveSyncConfig} constructor directly and provide the props, + * and set {@link HoodieSyncConfig#META_SYNC_BASE_PATH} and {@link HoodieSyncConfig#META_SYNC_BASE_FILE_FORMAT} instead. + */ + @Deprecated public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) { checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE().key())); HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); @@ -310,7 +316,7 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b hiveSyncConfig.isConditionalSync = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().key(), DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().defaultValue())); hiveSyncConfig.bucketSpec = props.getBoolean(DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().key(), - (boolean) DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue()) + DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue()) ? HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())) : null; if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 530e435783696..f86e55b436f1e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -25,9 +25,10 @@ import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.Option import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} import org.apache.hudi.hive.util.ConfigUtils -import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor} +import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor} import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} +import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.log4j.LogManager import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils} @@ -246,7 +247,7 @@ object DataSourceWriteOptions { } val TABLE_NAME: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.write.table.name") + .key(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY) .noDefaultValue() .withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.") @@ -380,185 +381,79 @@ object DataSourceWriteOptions { // HIVE SYNC SPECIFIC CONFIGS // NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes // unexpected issues with config getting reset - val HIVE_SYNC_ENABLED: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.enable") - .defaultValue("false") - .withDocumentation("When set to true, register/sync the table to Apache Hive metastore") - - val META_SYNC_ENABLED: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.meta.sync.enable") - .defaultValue("false") - .withDocumentation("") - - val HIVE_DATABASE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.database") - .defaultValue("default") - .withDocumentation("database to sync to") - - val hiveTableOptKeyInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { - if (p.contains(TABLE_NAME)) { - Option.of(p.getString(TABLE_NAME)) - } else if (p.contains(HoodieWriteConfig.TBL_NAME)) { - Option.of(p.getString(HoodieWriteConfig.TBL_NAME)) - } else { - Option.empty[String]() - } - }) - val HIVE_TABLE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.table") - .defaultValue("unknown") - .withInferFunction(hiveTableOptKeyInferFunc) - .withDocumentation("table to sync to") - - val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.base_file_format") - .defaultValue("PARQUET") - .withDocumentation("Base file format for the sync.") - - val HIVE_USER: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.username") - .defaultValue("hive") - .withDocumentation("hive user name to use") - - val HIVE_PASS: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.password") - .defaultValue("hive") - .withDocumentation("hive password to use") - - val HIVE_URL: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.jdbcurl") - .defaultValue("jdbc:hive2://localhost:10000") - .withDocumentation("Hive jdbc url") - - val METASTORE_URIS: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.metastore.uris") - .defaultValue("thrift://localhost:9083") - .withDocumentation("Hive metastore url") - - val hivePartitionFieldsInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { - if (p.contains(PARTITIONPATH_FIELD)) { - Option.of(p.getString(PARTITIONPATH_FIELD)) - } else { - Option.empty[String]() - } - }) - val HIVE_PARTITION_FIELDS: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.partition_fields") - .defaultValue("") - .withDocumentation("Field in the table to use for determining hive partition columns.") - .withInferFunction(hivePartitionFieldsInferFunc) - - val hivePartitionExtractorInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => { - if (!p.contains(PARTITIONPATH_FIELD)) { - Option.of(classOf[NonPartitionedExtractor].getName) - } else { - val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length - if (numOfPartFields == 1 && p.contains(HIVE_STYLE_PARTITIONING) && p.getString(HIVE_STYLE_PARTITIONING) == "true") { - Option.of(classOf[HiveStylePartitionValueExtractor].getName) - } else { - Option.of(classOf[MultiPartKeysValueExtractor].getName) - } - } - }) - val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.partition_extractor_class") - .defaultValue(classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName) - .withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, " - + "default 'SlashEncodedDayPartitionValueExtractor'.") - .withInferFunction(hivePartitionExtractorInferFunc) - - val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.assume_date_partitioning") - .defaultValue("false") - .withDocumentation("Assume partitioning is yyyy/mm/dd") - - val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.use_pre_apache_input_format") - .defaultValue("false") - .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " - + "Use this when you are in the process of migrating from " - + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format") + /** + * @deprecated Hive Specific Configs are moved to {@link HiveSyncConfig} + */ + @Deprecated + val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_ENABLED + @Deprecated + val META_SYNC_ENABLED: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ENABLED + @Deprecated + val HIVE_DATABASE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_DATABASE_NAME + @Deprecated + val hiveTableOptKeyInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.TABLE_NAME_INFERENCE_FUNCTION + @Deprecated + val HIVE_TABLE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_TABLE_NAME + @Deprecated + val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT + @Deprecated + val HIVE_USER: ConfigProperty[String] = HiveSyncConfig.HIVE_USER + @Deprecated + val HIVE_PASS: ConfigProperty[String] = HiveSyncConfig.HIVE_PASS + @Deprecated + val HIVE_URL: ConfigProperty[String] = HiveSyncConfig.HIVE_URL + @Deprecated + val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfig.METASTORE_URIS + @Deprecated + val hivePartitionFieldsInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_FIELDS_INFERENCE_FUNCTION + @Deprecated + val HIVE_PARTITION_FIELDS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS + @Deprecated + val hivePartitionExtractorInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_EXTRACTOR_CLASS_FUNCTION + @Deprecated + val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS + @Deprecated + val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION + @Deprecated + val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT /** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */ @Deprecated - val HIVE_USE_JDBC: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.use_jdbc") - .defaultValue("true") - .deprecatedAfter("0.9.0") - .withDocumentation("Use JDBC when hive synchronization is enabled") - - val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.auto_create_database") - .defaultValue("true") - .withDocumentation("Auto create hive database if does not exists") - - val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.ignore_exceptions") - .defaultValue("false") - .withDocumentation("") - - val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.skip_ro_suffix") - .defaultValue("false") - .withDocumentation("Skip the _ro suffix for Read optimized table, when registering") - - val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.support_timestamp") - .defaultValue("false") - .withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " + - "Disabled by default for backward compatibility.") + val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_JDBC + @Deprecated + val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE + @Deprecated + val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS + @Deprecated + val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE + @Deprecated + val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE /** * Flag to indicate whether to use conditional syncing in HiveSync. * If set true, the Hive sync procedure will only run if partition or schema changes are detected. * By default true. */ - val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.conditional_sync") - .defaultValue("false") - .withDocumentation("Enables conditional hive sync, where partition or schema change must exist to perform sync to hive.") - - val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.table_properties") - .noDefaultValue() - .withDocumentation("Additional properties to store with table.") - - val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.serde_properties") - .noDefaultValue() - .withDocumentation("Serde properties to hive table.") - - val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.sync_as_datasource") - .defaultValue("true") - .withDocumentation("") + @Deprecated + val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC + @Deprecated + val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_PROPERTIES + @Deprecated + val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES + @Deprecated + val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE // Create table as managed table - val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[Boolean] = ConfigProperty - .key("hoodie.datasource.hive_sync.create_managed_table") - .defaultValue(false) - .withDocumentation("Whether to sync the table as managed table.") - - val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[Int] = ConfigProperty - .key("hoodie.datasource.hive_sync.batch_num") - .defaultValue(1000) - .withDocumentation("The number of partitions one batch when synchronous partitions to hive.") - - val HIVE_SYNC_MODE: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.mode") - .noDefaultValue() - .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.") - - val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[Boolean] = ConfigProperty - .key("hoodie.datasource.hive_sync.bucket_sync") - .defaultValue(false) - .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." + - "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'") - - val HIVE_SYNC_COMMENT: ConfigProperty[String] = ConfigProperty - .key("hoodie.datasource.hive_sync.sync_comment") - .defaultValue("false") - .withDocumentation("Whether to sync the table column comments while syncing the table.") + @Deprecated + val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE + @Deprecated + val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM + @Deprecated + val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_MODE + @Deprecated + val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC + @Deprecated + val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_COMMENT; // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty @@ -584,19 +479,19 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */ @Deprecated - val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HIVE_ASSUME_DATE_PARTITION.key() + val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key() /** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */ @Deprecated - val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() + val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ @Deprecated - val HIVE_USE_JDBC_OPT_KEY = HIVE_USE_JDBC.key() + val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.key() + val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.key() + val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key() /** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */ @Deprecated val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = STREAMING_IGNORE_FAILED_BATCH.key() @@ -611,34 +506,34 @@ object DataSourceWriteOptions { val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue() /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */ @Deprecated - val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key() + val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfig.HIVE_SYNC_ENABLED.key() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated - val META_SYNC_ENABLED_OPT_KEY = META_SYNC_ENABLED.key() + val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() /** @deprecated Use {@link HIVE_DATABASE} and its methods instead */ @Deprecated - val HIVE_DATABASE_OPT_KEY = HIVE_DATABASE.key() + val HIVE_DATABASE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() /** @deprecated Use {@link HIVE_TABLE} and its methods instead */ @Deprecated - val HIVE_TABLE_OPT_KEY = HIVE_TABLE.key() + val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() /** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */ @Deprecated - val HIVE_BASE_FILE_FORMAT_OPT_KEY = HIVE_BASE_FILE_FORMAT.key() + val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val HIVE_USER_OPT_KEY = HIVE_USER.key() + val HIVE_USER_OPT_KEY = HiveSyncConfig.HIVE_USER.key() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val HIVE_PASS_OPT_KEY = HIVE_PASS.key() + val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val HIVE_URL_OPT_KEY = HIVE_URL.key() + val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated - val HIVE_PARTITION_FIELDS_OPT_KEY = HIVE_PARTITION_FIELDS.key() + val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key() /** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */ @Deprecated - val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HIVE_PARTITION_EXTRACTOR_CLASS.key() + val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key() /** @deprecated Use {@link KEYGENERATOR_CLASS_NAME} and its methods instead */ @Deprecated @@ -748,60 +643,60 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HIVE_SYNC_ENABLED.defaultValue() + val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated - val DEFAULT_META_SYNC_ENABLED_OPT_VAL = META_SYNC_ENABLED.defaultValue() + val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue() /** @deprecated Use {@link HIVE_DATABASE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_DATABASE_OPT_VAL = HIVE_DATABASE.defaultValue() + val DEFAULT_HIVE_DATABASE_OPT_VAL = HoodieSyncConfig.META_SYNC_DATABASE_NAME.defaultValue() /** @deprecated Use {@link HIVE_TABLE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_TABLE_OPT_VAL = HIVE_TABLE.defaultValue() + val DEFAULT_HIVE_TABLE_OPT_VAL = HoodieSyncConfig.META_SYNC_TABLE_NAME.defaultValue() /** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */ @Deprecated - val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HIVE_BASE_FILE_FORMAT.defaultValue() + val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val DEFAULT_HIVE_USER_OPT_VAL = HIVE_USER.defaultValue() + val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfig.HIVE_USER.defaultValue() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PASS_OPT_VAL = HIVE_PASS.defaultValue() + val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val DEFAULT_HIVE_URL_OPT_VAL = HIVE_URL.defaultValue() + val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HIVE_PARTITION_FIELDS.defaultValue() + val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue() /** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HIVE_PARTITION_EXTRACTOR_CLASS.defaultValue() + val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.defaultValue() /** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */ @Deprecated - val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HIVE_ASSUME_DATE_PARTITION.defaultValue() + val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue() @Deprecated val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ @Deprecated - val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HIVE_USE_JDBC.defaultValue() + val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfig.HIVE_USE_JDBC.defaultValue() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.defaultValue() + val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.defaultValue() + val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue() /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */ @Deprecated - val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() + val HIVE_SKIP_RO_SUFFIX = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() + val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key() + val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() + val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() /** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */ @Deprecated val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 8ba0db907b370..fc83cebc945d4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -19,7 +19,6 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord -import org.apache.avro.reflect.AvroSchema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -32,7 +31,7 @@ import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model._ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils} +import org.apache.hudi.common.util.{CommitUtils, StringUtils} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.exception.HoodieException @@ -40,24 +39,21 @@ import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRo import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} import org.apache.hudi.index.SparkHoodieIndexFactory import org.apache.hudi.internal.DataSourceInternalWriterHelper -import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory -import org.apache.hudi.sync.common.AbstractSyncTool +import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} +import org.apache.hudi.sync.common.HoodieSyncConfig +import org.apache.hudi.sync.common.util.SyncUtilHelpers import org.apache.hudi.table.BulkInsertPartitioner import org.apache.log4j.LogManager -import org.apache.spark.SPARK_VERSION import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.rdd.RDD -import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} -import org.apache.spark.sql.types.StructType import org.apache.spark.sql._ -import org.apache.spark.SparkContext +import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.types.StructType +import org.apache.spark.{SPARK_VERSION, SparkContext} -import java.util.Properties import scala.collection.JavaConversions._ import scala.collection.mutable -import scala.collection.mutable.ListBuffer object HoodieSparkSqlWriter { @@ -512,14 +508,7 @@ object HoodieSparkSqlWriter { + " To use row writer please switch to spark 2 or spark 3") } val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params) - val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean - val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean - val syncHiveSuccess = - if (hiveSyncEnabled || metaSyncEnabled) { - metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) - } else { - true - } + val syncHiveSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) (syncHiveSuccess, common.util.Option.ofNullable(instantTime)) } @@ -558,57 +547,10 @@ object HoodieSparkSqlWriter { } } - private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig, sqlConf: SQLConf): Boolean = { - val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig, sqlConf) - val hiveConf: HiveConf = new HiveConf() - hiveConf.addResource(fs.getConf) - if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) { - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris) - } - new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable() - true - } - - private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig, sqlConf: SQLConf): HiveSyncConfig = { - val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig() - hiveSyncConfig.basePath = basePath.toString - hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT) - hiveSyncConfig.usePreApacheInputFormat = - hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT).toBoolean - hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE) - hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE) - hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER) - hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS) - hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL) - hiveSyncConfig.metastoreUris = hoodieConfig.getStringOrDefault(METASTORE_URIS) - hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE, - DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean - hiveSyncConfig.partitionFields = - ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*) - hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS) - hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC) - hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE) - hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean - hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean - hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean - hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean - hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt - - hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean - hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD) - hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE) - hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE) - hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES) - hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES) - hiveSyncConfig.sparkVersion = SPARK_VERSION - hiveSyncConfig.syncComment = hoodieConfig.getStringOrDefault(HIVE_SYNC_COMMENT).toBoolean - hiveSyncConfig - } - private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path, schema: StructType): Boolean = { - val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean - var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean + val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean + var metaSyncEnabled = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_ENABLED).toBoolean var syncClientToolClassSet = scala.collection.mutable.Set[String]() hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass) @@ -617,29 +559,23 @@ object HoodieSparkSqlWriter { metaSyncEnabled = true syncClientToolClassSet += classOf[HiveSyncTool].getName } - var metaSyncSuccess = true + if (metaSyncEnabled) { val fs = basePath.getFileSystem(spark.sessionState.newHadoopConf()) + val properties = new TypedProperties() + properties.putAll(hoodieConfig.getProps) + properties.put(HiveSyncConfig.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key, spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString) + properties.put(HoodieSyncConfig.META_SYNC_SPARK_VERSION.key, SPARK_VERSION) + properties.put(HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA.key, hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)) + + val hiveConf: HiveConf = new HiveConf() + hiveConf.addResource(fs.getConf) + syncClientToolClassSet.foreach(impl => { - val syncSuccess = impl.trim match { - case "org.apache.hudi.hive.HiveSyncTool" => { - log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL) + ")") - syncHive(basePath, fs, hoodieConfig, spark.sessionState.conf) - true - } - case _ => { - val properties = new Properties() - properties.putAll(hoodieConfig.getProps) - properties.put("basePath", basePath.toString) - val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool] - syncHoodie.syncHoodieTable() - true - } - } - metaSyncSuccess = metaSyncSuccess && syncSuccess + SyncUtilHelpers.runHoodieMetaSync(impl.trim, properties, hiveConf, fs, basePath.toString, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue) }) } - metaSyncSuccess + true } /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 8a4ad9d85d72d..a4d76763fc558 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -18,13 +18,14 @@ package org.apache.hudi import java.util.Properties - import org.apache.hudi.DataSourceOptionsHelper.allAlternatives import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.exception.HoodieException +import org.apache.hudi.hive.HiveSyncConfig +import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hudi.command.SqlKeyGenerator @@ -64,21 +65,21 @@ object HoodieWriterUtils { hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS) hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH) hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME) - hoodieConfig.setDefaultValue(HIVE_SYNC_ENABLED) - hoodieConfig.setDefaultValue(META_SYNC_ENABLED) - hoodieConfig.setDefaultValue(HIVE_DATABASE) - hoodieConfig.setDefaultValue(HIVE_TABLE) - hoodieConfig.setDefaultValue(HIVE_BASE_FILE_FORMAT) - hoodieConfig.setDefaultValue(HIVE_USER) - hoodieConfig.setDefaultValue(HIVE_PASS) - hoodieConfig.setDefaultValue(HIVE_URL) - hoodieConfig.setDefaultValue(METASTORE_URIS) - hoodieConfig.setDefaultValue(HIVE_PARTITION_FIELDS) - hoodieConfig.setDefaultValue(HIVE_PARTITION_EXTRACTOR_CLASS) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_ENABLED) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_ENABLED) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT) + hoodieConfig.setDefaultValue(HiveSyncConfig.METASTORE_URIS) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USER) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_PASS) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_URL) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS) + hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING) - hoodieConfig.setDefaultValue(HIVE_USE_JDBC) - hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE) - hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE) + hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE) hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE) hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala index 7f3cedbd3986e..2e639d78e1e17 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala @@ -22,8 +22,9 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME -import org.apache.hudi.hive.MultiPartKeysValueExtractor +import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor} import org.apache.hudi.hive.ddl.HiveSyncMode +import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver @@ -102,15 +103,15 @@ case class AlterHoodieTableDropPartitionCommand( RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), PARTITIONPATH_FIELD.key -> partitionFields, - HIVE_SYNC_ENABLED.key -> enableHive.toString, - META_SYNC_ENABLED.key -> enableHive.toString, - HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), - HIVE_USE_JDBC.key -> "false", - HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"), - HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table, - HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", - HIVE_PARTITION_FIELDS.key -> partitionFields, - HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName + HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), + HiveSyncConfig.HIVE_USE_JDBC.key -> "false", + HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"), + HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hoodieCatalogTable.table.identifier.table, + HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", + HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields, + HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 2877dd8d9ee94..1d2cea10afa7d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions +import org.apache.hudi.hive.HiveSyncConfig import org.apache.hudi.hive.util.ConfigUtils import org.apache.hudi.sql.InsertMode import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, HoodieCatalogTable} @@ -80,9 +81,9 @@ case class CreateHoodieTableAsSelectCommand( val tblProperties = hoodieCatalogTable.catalogProperties val options = Map( - DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString, - DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava), - DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava), + HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString, + HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava), + HiveSyncConfig.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava), DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala index fff8f9194562f..f6da1b3868302 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME +import org.apache.hudi.hive.HiveSyncConfig import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} import org.apache.spark.sql._ @@ -75,8 +76,8 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Hoodie SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, - HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), - HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", + HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), + HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 1d9aedd2af6fa..74d6226b49f6d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -22,8 +22,9 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME -import org.apache.hudi.hive.MultiPartKeysValueExtractor +import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor} import org.apache.hudi.hive.ddl.HiveSyncMode +import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, SparkAdapterSupport} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier @@ -462,14 +463,14 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning, KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName, SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName, - META_SYNC_ENABLED.key -> enableHive.toString, - HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), - HIVE_USE_JDBC.key -> "false", - HIVE_DATABASE.key -> targetTableDb, - HIVE_TABLE.key -> targetTableName, - HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", - HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, - HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, + HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), + HiveSyncConfig.HIVE_USE_JDBC.key -> "false", + HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb, + HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName, + HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", + HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, + HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index b6e595c40a8df..d25896bdd9779 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.hive.NonPartitionedExtractor; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; @@ -255,24 +256,24 @@ public void run() throws Exception { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable) - .option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB) - .option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl) - .option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser) - .option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass) - .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true"); + writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable) + .option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB) + .option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl) + .option(HiveSyncConfig.HIVE_USER.key(), hiveUser) + .option(HiveSyncConfig.HIVE_PASS.key(), hivePass) + .option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"); if (nonPartitionedTable) { writer = writer - .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + .option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()) .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), ""); } else if (useMultiPartitionKeys) { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java index 8302ece4b9ae9..42c6a4fd89f41 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.hive.NonPartitionedExtractor; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; @@ -125,24 +126,24 @@ private HoodieTestDataGenerator getDataGenerate() { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable) - .option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB) - .option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl) - .option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser) - .option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass) - .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true"); + writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable) + .option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB) + .option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl) + .option(HiveSyncConfig.HIVE_USER.key(), hiveUser) + .option(HiveSyncConfig.HIVE_PASS.key(), hivePass) + .option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"); if (nonPartitionedTable) { writer = writer - .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + .option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()) .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), ""); } else if (useMultiPartitionKeys) { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 4af3943966aa3..207a9492fdad0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -27,6 +27,7 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; @@ -382,19 +383,19 @@ public void stream(Dataset streamingInput, String operationType, String che private DataStreamWriter updateHiveSyncConfig(DataStreamWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable) - .option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB) - .option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl) - .option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser) - .option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass) - .option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true"); + writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable) + .option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB) + .option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl) + .option(HiveSyncConfig.HIVE_USER.key(), hiveUser) + .option(HiveSyncConfig.HIVE_PASS.key(), hivePass) + .option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"); if (useMultiPartitionKeys) { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( - DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( + HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala index d5c3bfa01fc2e..9920aa80baf09 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala @@ -20,6 +20,7 @@ package org.apache.hudi import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, MultiPartKeysValueExtractor} import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} +import org.apache.hudi.sync.common.HoodieSyncConfig import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test @@ -31,10 +32,10 @@ class TestDataSourceOptions { ) val modifiedOptions1 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions1) assertEquals(classOf[ComplexKeyGenerator].getName, modifiedOptions1(KEYGENERATOR_CLASS_NAME.key)) - assertEquals("hudi_table", modifiedOptions1(HIVE_TABLE.key)) - assertEquals("year,month", modifiedOptions1(HIVE_PARTITION_FIELDS.key)) + assertEquals("hudi_table", modifiedOptions1(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) + assertEquals("year,month", modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key)) assertEquals(classOf[MultiPartKeysValueExtractor].getName, - modifiedOptions1(HIVE_PARTITION_EXTRACTOR_CLASS.key)) + modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key)) val inputOptions2 = Map( TABLE_NAME.key -> "hudi_table", @@ -43,9 +44,9 @@ class TestDataSourceOptions { ) val modifiedOptions2 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions2) assertEquals(classOf[SimpleKeyGenerator].getName, modifiedOptions2(KEYGENERATOR_CLASS_NAME.key)) - assertEquals("hudi_table", modifiedOptions2(HIVE_TABLE.key)) - assertEquals("year", modifiedOptions2(HIVE_PARTITION_FIELDS.key)) + assertEquals("hudi_table", modifiedOptions2(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) + assertEquals("year", modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key)) assertEquals(classOf[HiveStylePartitionValueExtractor].getName, - modifiedOptions2(HIVE_PARTITION_EXTRACTOR_CLASS.key)) + modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key)) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index b5186fb1ac089..c14d0bb063d7b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -664,55 +664,6 @@ class TestHoodieSparkSqlWriter { assertEquals(expectedSchema, actualSchema) } - /** - * Test case for build sync config for spark sql. - */ - @Test - def testBuildSyncConfigForSparkSql(): Unit = { - val params = Map( - "path" -> tempBasePath, - DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie", - DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition", - DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key -> "true", - DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> "true" - ) - val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params) - val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters) - - val buildSyncConfigMethod = - HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path], - classOf[HoodieConfig], classOf[SQLConf]) - buildSyncConfigMethod.setAccessible(true) - - val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter, - new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig] - assertTrue(hiveSyncConfig.skipROSuffix) - assertTrue(hiveSyncConfig.createManagedTable) - assertTrue(hiveSyncConfig.syncAsSparkDataSourceTable) - assertResult(spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD))(hiveSyncConfig.sparkSchemaLengthThreshold) - } - - /** - * Test case for build sync config for skip Ro Suffix values. - */ - @Test - def testBuildSyncConfigForSkipRoSuffixValues(): Unit = { - val params = Map( - "path" -> tempBasePath, - DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie", - DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition" - ) - val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params) - val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters) - val buildSyncConfigMethod = - HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path], - classOf[HoodieConfig], classOf[SQLConf]) - buildSyncConfigMethod.setAccessible(true) - val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter, - new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig] - assertFalse(hiveSyncConfig.skipROSuffix) - } - /** * Test case for incremental view with replacement. */ 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 bf0369ae2ee58..2088d48d8a383 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 @@ -23,6 +23,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; + +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.util.Option; @@ -41,7 +43,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.stream.Collectors; /** @@ -63,8 +64,8 @@ public class DLASyncTool extends AbstractSyncTool { private final String snapshotTableName; private final Option roTableTableName; - public DLASyncTool(Properties properties, FileSystem fs) { - super(properties, fs); + public DLASyncTool(TypedProperties properties, Configuration conf, FileSystem fs) { + super(properties, conf, fs); this.hoodieDLAClient = new HoodieDLAClient(Utils.propertiesToConfig(properties), fs); this.cfg = Utils.propertiesToConfig(properties); switch (hoodieDLAClient.getTableType()) { @@ -205,7 +206,8 @@ public static void main(String[] args) { cmd.usage(); System.exit(1); } - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - new DLASyncTool(Utils.configToProperties(cfg), fs).syncHoodieTable(); + Configuration hadoopConf = new Configuration(); + FileSystem fs = FSUtils.getFs(cfg.basePath, hadoopConf); + new DLASyncTool(Utils.configToProperties(cfg), hadoopConf, fs).syncHoodieTable(); } } 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 ad47b71f84948..d1b0dd4e9d56f 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 @@ -18,12 +18,12 @@ package org.apache.hudi.dla.util; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.dla.DLASyncConfig; import java.util.ArrayList; import java.util.Arrays; -import java.util.Properties; public class Utils { public static String DLA_DATABASE_OPT_KEY = "hoodie.datasource.dla_sync.database"; @@ -39,8 +39,8 @@ public class Utils { 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) { - Properties properties = new Properties(); + public static TypedProperties configToProperties(DLASyncConfig cfg) { + TypedProperties properties = new TypedProperties(); properties.put(DLA_DATABASE_OPT_KEY, cfg.databaseName); properties.put(DLA_TABLE_OPT_KEY, cfg.tableName); properties.put(DLA_USER_OPT_KEY, cfg.dlaUser); @@ -54,7 +54,7 @@ public static Properties configToProperties(DLASyncConfig cfg) { return properties; } - public static DLASyncConfig propertiesToConfig(Properties properties) { + public static DLASyncConfig propertiesToConfig(TypedProperties properties) { DLASyncConfig config = new DLASyncConfig(); config.databaseName = properties.getProperty(DLA_DATABASE_OPT_KEY); config.tableName = properties.getProperty(DLA_TABLE_OPT_KEY); 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 8ee9daa836648..eb4fc62d40d6d 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,27 +18,16 @@ package org.apache.hudi.hive; -import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.sync.common.HoodieSyncConfig; import com.beust.jcommander.Parameter; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - /** - * Configs needed to sync data into Hive. + * Configs needed to sync data into the Hive Metastore. */ -public class HiveSyncConfig implements Serializable { - - @Parameter(names = {"--database"}, description = "name of the target database in Hive", required = true) - public String databaseName; - - @Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true) - public String tableName; - - @Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)") - public String baseFileFormat = "PARQUET"; +public class HiveSyncConfig extends HoodieSyncConfig { @Parameter(names = {"--user"}, description = "Hive username") public String hiveUser; @@ -52,48 +41,31 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris") public String metastoreUris; - @Parameter(names = {"--base-path"}, description = "Basepath of hoodie table to sync", required = true) - public String basePath; - - @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by") - public List partitionFields = new ArrayList<>(); - - @Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor " - + "to extract the partition values from HDFS path") - public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class.getName(); - - @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" - + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") - public Boolean assumeDatePartitioning = false; - @Parameter(names = {"--use-pre-apache-input-format"}, description = "Use InputFormat under com.uber.hoodie package " + "instead of org.apache.hudi package. Use this when you are in the process of migrating from " + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to " + "org.apache.hudi input format.") - public Boolean usePreApacheInputFormat = false; + public Boolean usePreApacheInputFormat; @Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in metastore", required = false) public String bucketSpec; @Deprecated @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") - public Boolean useJdbc = true; + public Boolean useJdbc; @Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql") public String syncMode; @Parameter(names = {"--auto-create-database"}, description = "Auto create hive database") - public Boolean autoCreateDatabase = true; + public Boolean autoCreateDatabase; @Parameter(names = {"--ignore-exceptions"}, description = "Ignore hive exceptions") - public Boolean ignoreExceptions = false; + public Boolean ignoreExceptions; @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; + public Boolean skipROSuffix; @Parameter(names = {"--table-properties"}, description = "Table properties to hive table") public String tableProperties; @@ -106,64 +78,170 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--support-timestamp"}, description = "'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type." + "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 Boolean supportTimestamp; @Parameter(names = {"--managed-table"}, description = "Create a managed table") - public Boolean createManagedTable = false; + public Boolean createManagedTable; @Parameter(names = {"--batch-sync-num"}, description = "The number of partitions one batch when synchronous partitions to hive") - public Integer batchSyncNum = 1000; + public Integer batchSyncNum; @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table.") - public Boolean syncAsSparkDataSourceTable = true; + public Boolean syncAsSparkDataSourceTable; @Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore.") - public int sparkSchemaLengthThreshold = 4000; + public int sparkSchemaLengthThreshold; @Parameter(names = {"--with-operation-field"}, description = "Whether to include the '_hoodie_operation' field in the metadata fields") public Boolean withOperationField = false; - @Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.") - public Boolean isConditionalSync = false; - - @Parameter(names = {"--spark-version"}, description = "The spark version", required = false) - public String sparkVersion; - @Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive") public boolean syncComment = false; - // enhance the similar function in child class - public static HiveSyncConfig copy(HiveSyncConfig cfg) { - HiveSyncConfig newConfig = new HiveSyncConfig(); - newConfig.basePath = cfg.basePath; - newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; - newConfig.databaseName = cfg.databaseName; - newConfig.hivePass = cfg.hivePass; - newConfig.hiveUser = cfg.hiveUser; - newConfig.partitionFields = cfg.partitionFields; - newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; - newConfig.jdbcUrl = cfg.jdbcUrl; - newConfig.metastoreUris = cfg.metastoreUris; - newConfig.tableName = cfg.tableName; - newConfig.bucketSpec = cfg.bucketSpec; - newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; - newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.supportTimestamp = cfg.supportTimestamp; - newConfig.decodePartition = cfg.decodePartition; - newConfig.tableProperties = cfg.tableProperties; - newConfig.serdeProperties = cfg.serdeProperties; - newConfig.createManagedTable = cfg.createManagedTable; - newConfig.batchSyncNum = cfg.batchSyncNum; - newConfig.syncAsSparkDataSourceTable = cfg.syncAsSparkDataSourceTable; - newConfig.sparkSchemaLengthThreshold = cfg.sparkSchemaLengthThreshold; - newConfig.withOperationField = cfg.withOperationField; - newConfig.isConditionalSync = cfg.isConditionalSync; - newConfig.sparkVersion = cfg.sparkVersion; - newConfig.syncComment = cfg.syncComment; - return newConfig; + // HIVE SYNC SPECIFIC CONFIGS + // NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes + // unexpected issues with config getting reset + public static final ConfigProperty HIVE_SYNC_ENABLED = ConfigProperty + .key("hoodie.datasource.hive_sync.enable") + .defaultValue("false") + .withDocumentation("When set to true, register/sync the table to Apache Hive metastore."); + + public static final ConfigProperty HIVE_USER = ConfigProperty + .key("hoodie.datasource.hive_sync.username") + .defaultValue("hive") + .withDocumentation("hive user name to use"); + + public static final ConfigProperty HIVE_PASS = ConfigProperty + .key("hoodie.datasource.hive_sync.password") + .defaultValue("hive") + .withDocumentation("hive password to use"); + + public static final ConfigProperty HIVE_URL = ConfigProperty + .key("hoodie.datasource.hive_sync.jdbcurl") + .defaultValue("jdbc:hive2://localhost:10000") + .withDocumentation("Hive metastore url"); + + public static final ConfigProperty HIVE_USE_PRE_APACHE_INPUT_FORMAT = ConfigProperty + .key("hoodie.datasource.hive_sync.use_pre_apache_input_format") + .defaultValue("false") + .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " + + "Use this when you are in the process of migrating from " + + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format"); + + /** + * @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0 + */ + @Deprecated + public static final ConfigProperty HIVE_USE_JDBC = ConfigProperty + .key("hoodie.datasource.hive_sync.use_jdbc") + .defaultValue("true") + .deprecatedAfter("0.9.0") + .withDocumentation("Use JDBC when hive synchronization is enabled"); + + public static final ConfigProperty METASTORE_URIS = ConfigProperty + .key("hoodie.datasource.hive_sync.metastore.uris") + .defaultValue("thrift://localhost:9083") + .withDocumentation("Hive metastore url"); + + public static final ConfigProperty HIVE_AUTO_CREATE_DATABASE = ConfigProperty + .key("hoodie.datasource.hive_sync.auto_create_database") + .defaultValue("true") + .withDocumentation("Auto create hive database if does not exists"); + + public static final ConfigProperty HIVE_IGNORE_EXCEPTIONS = ConfigProperty + .key("hoodie.datasource.hive_sync.ignore_exceptions") + .defaultValue("false") + .withDocumentation("Ignore exceptions when syncing with Hive."); + + public static final ConfigProperty HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.skip_ro_suffix") + .defaultValue("false") + .withDocumentation("Skip the _ro suffix for Read optimized table, when registering"); + + public static final ConfigProperty HIVE_SUPPORT_TIMESTAMP_TYPE = ConfigProperty + .key("hoodie.datasource.hive_sync.support_timestamp") + .defaultValue("false") + .withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " + + "Disabled by default for backward compatibility."); + + public static final ConfigProperty HIVE_TABLE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.hive_sync.table_properties") + .noDefaultValue() + .withDocumentation("Additional properties to store with table."); + + public static final ConfigProperty HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.hive_sync.serde_properties") + .noDefaultValue() + .withDocumentation("Serde properties to hive table."); + + public static final ConfigProperty HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.sync_as_datasource") + .defaultValue("true") + .withDocumentation(""); + + public static final ConfigProperty HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD = ConfigProperty + .key("hoodie.datasource.hive_sync.schema_string_length_thresh") + .defaultValue(4000) + .withDocumentation(""); + + // Create table as managed table + public static final ConfigProperty HIVE_CREATE_MANAGED_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.create_managed_table") + .defaultValue(false) + .withDocumentation("Whether to sync the table as managed table."); + + public static final ConfigProperty HIVE_BATCH_SYNC_PARTITION_NUM = ConfigProperty + .key("hoodie.datasource.hive_sync.batch_num") + .defaultValue(1000) + .withDocumentation("The number of partitions one batch when synchronous partitions to hive."); + + public static final ConfigProperty HIVE_SYNC_MODE = ConfigProperty + .key("hoodie.datasource.hive_sync.mode") + .noDefaultValue() + .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql."); + + public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC = ConfigProperty + .key("hoodie.datasource.hive_sync.bucket_sync") + .defaultValue(false) + .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." + + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); + + public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC_SPEC = ConfigProperty + .key("hoodie.datasource.hive_sync.bucket_sync_spec") + .defaultValue("") + .withDocumentation("The hive metastore bucket specification when using bucket index." + + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); + + public static final ConfigProperty HIVE_SYNC_COMMENT = ConfigProperty + .key("hoodie.datasource.hive_sync.sync_comment") + .defaultValue("false") + .withDocumentation("Whether to sync the table column comments while syncing the table."); + + public HiveSyncConfig() { + this(new TypedProperties()); + } + + public HiveSyncConfig(TypedProperties props) { + super(props); + this.hiveUser = getStringOrDefault(HIVE_USER); + this.hivePass = getStringOrDefault(HIVE_PASS); + this.jdbcUrl = getStringOrDefault(HIVE_URL); + this.usePreApacheInputFormat = getBooleanOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT); + this.useJdbc = getBooleanOrDefault(HIVE_USE_JDBC); + this.metastoreUris = getStringOrDefault(METASTORE_URIS); + this.syncMode = getString(HIVE_SYNC_MODE); + this.autoCreateDatabase = getBooleanOrDefault(HIVE_AUTO_CREATE_DATABASE); + this.ignoreExceptions = getBooleanOrDefault(HIVE_IGNORE_EXCEPTIONS); + this.skipROSuffix = getBooleanOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE); + this.tableProperties = getString(HIVE_TABLE_PROPERTIES); + this.serdeProperties = getString(HIVE_TABLE_SERDE_PROPERTIES); + this.supportTimestamp = getBooleanOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE); + this.batchSyncNum = getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM); + this.syncAsSparkDataSourceTable = getBooleanOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE); + this.sparkSchemaLengthThreshold = getIntOrDefault(HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD); + this.createManagedTable = getBooleanOrDefault(HIVE_CREATE_MANAGED_TABLE); + this.bucketSpec = getStringOrDefault(HIVE_SYNC_BUCKET_SYNC_SPEC); + this.syncComment = getBooleanOrDefault(HIVE_SYNC_COMMENT); } @Override @@ -197,6 +275,7 @@ public String toString() { + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold + ", withOperationField=" + withOperationField + ", isConditionalSync=" + isConditionalSync + + ", sparkVersion=" + sparkVersion + ", syncComment=" + syncComment + '}'; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 952742b913330..cac70ab5446e4 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -25,6 +25,8 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; + +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; @@ -70,40 +72,53 @@ public class HiveSyncTool extends AbstractSyncTool { public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - protected final HiveSyncConfig cfg; + protected final HiveSyncConfig hiveSyncConfig; protected HoodieHiveClient hoodieHiveClient = null; protected String snapshotTableName = null; protected Option roTableName = null; - public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(configuration.getAllProperties(), fs); + public HiveSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { + super(props, conf, fs); + this.hiveSyncConfig = new HiveSyncConfig(props); + init(hiveSyncConfig, new HiveConf(conf, HiveConf.class)); + } + + @Deprecated + public HiveSyncTool(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf, FileSystem fs) { + super(hiveSyncConfig.getProps(), hiveConf, fs); + this.hiveSyncConfig = hiveSyncConfig; + init(hiveSyncConfig, hiveConf); + } + private void init(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { try { - this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs); + if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) { + hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris); + } + this.hoodieHiveClient = new HoodieHiveClient(hiveSyncConfig, hiveConf, fs); } catch (RuntimeException e) { - if (cfg.ignoreExceptions) { + if (hiveSyncConfig.ignoreExceptions) { LOG.error("Got runtime exception when hive syncing, but continuing as ignoreExceptions config is set ", e); } else { throw new HoodieHiveSyncException("Got runtime exception when hive syncing", e); } } - this.cfg = cfg; // Set partitionFields to empty, when the NonPartitionedExtractor is used - if (NonPartitionedExtractor.class.getName().equals(cfg.partitionValueExtractorClass)) { + if (NonPartitionedExtractor.class.getName().equals(hiveSyncConfig.partitionValueExtractorClass)) { LOG.warn("Set partitionFields to empty, since the NonPartitionedExtractor is used"); - cfg.partitionFields = new ArrayList<>(); + hiveSyncConfig.partitionFields = new ArrayList<>(); } if (hoodieHiveClient != null) { switch (hoodieHiveClient.getTableType()) { case COPY_ON_WRITE: - this.snapshotTableName = cfg.tableName; + this.snapshotTableName = hiveSyncConfig.tableName; this.roTableName = Option.empty(); break; case MERGE_ON_READ: - this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE; - this.roTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) : - Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + this.snapshotTableName = hiveSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableName = hiveSyncConfig.skipROSuffix ? Option.of(hiveSyncConfig.tableName) : + Option.of(hiveSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE); break; default: LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); @@ -116,10 +131,13 @@ public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { public void syncHoodieTable() { try { if (hoodieHiveClient != null) { + LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :" + + hiveSyncConfig.jdbcUrl + ", basePath :" + hiveSyncConfig.basePath); + doSync(); } } catch (RuntimeException re) { - throw new HoodieException("Got runtime exception when hive syncing " + cfg.tableName, re); + throw new HoodieException("Got runtime exception when hive syncing " + hiveSyncConfig.tableName, re); } finally { if (hoodieHiveClient != null) { hoodieHiveClient.close(); @@ -150,18 +168,19 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, + " of type " + hoodieHiveClient.getTableType()); // check if the database exists else create it - if (cfg.autoCreateDatabase) { + if (hiveSyncConfig.autoCreateDatabase) { try { - if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) { - hoodieHiveClient.createDatabase(cfg.databaseName); + if (!hoodieHiveClient.doesDataBaseExist(hiveSyncConfig.databaseName)) { + hoodieHiveClient.createDatabase(hiveSyncConfig.databaseName); } } catch (Exception e) { // this is harmless since table creation will fail anyways, creation of DB is needed for in-memory testing LOG.warn("Unable to create database", e); } } else { - if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) { - throw new HoodieHiveSyncException("hive database does not exist " + cfg.databaseName); + if (!hoodieHiveClient.doesDataBaseExist(hiveSyncConfig.databaseName)) { + LOG.error("Hive database does not exist " + hiveSyncConfig.databaseName); + throw new HoodieHiveSyncException("hive database does not exist " + hiveSyncConfig.databaseName); } } @@ -181,7 +200,7 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, if (hoodieHiveClient.isBootstrap() && hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ && !readAsOptimized) { - cfg.syncAsSparkDataSourceTable = false; + hiveSyncConfig.syncAsSparkDataSourceTable = false; } // Sync schema if needed @@ -200,7 +219,7 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Sync the partitions if needed boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition); boolean meetSyncConditions = schemaChanged || partitionsChanged; - if (!cfg.isConditionalSync || meetSyncConditions) { + if (!hiveSyncConfig.isConditionalSync || meetSyncConditions) { hoodieHiveClient.updateLastCommitTimeSynced(tableName); } LOG.info("Sync complete for " + tableName); @@ -216,10 +235,10 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, private boolean syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, boolean readAsOptimized, MessageType schema) { // Append spark table properties & serde properties - Map tableProperties = ConfigUtils.toMap(cfg.tableProperties); - Map serdeProperties = ConfigUtils.toMap(cfg.serdeProperties); - if (cfg.syncAsSparkDataSourceTable) { - Map sparkTableProperties = getSparkTableProperties(cfg.sparkSchemaLengthThreshold, schema); + Map tableProperties = ConfigUtils.toMap(hiveSyncConfig.tableProperties); + Map serdeProperties = ConfigUtils.toMap(hiveSyncConfig.serdeProperties); + if (hiveSyncConfig.syncAsSparkDataSourceTable) { + Map sparkTableProperties = getSparkTableProperties(hiveSyncConfig.sparkSchemaLengthThreshold, schema); Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized); tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); @@ -228,10 +247,10 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea // Check and sync schema if (!tableExists) { LOG.info("Hive table " + tableName + " is not found. Creating it"); - HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(cfg.baseFileFormat.toUpperCase()); + HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(hiveSyncConfig.baseFileFormat.toUpperCase()); String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat); - if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && cfg.usePreApacheInputFormat) { + if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && hiveSyncConfig.usePreApacheInputFormat) { // Parquet input format had an InputFormat class visible under the old naming scheme. inputFormatClassName = useRealTimeInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName() @@ -250,12 +269,12 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea } else { // Check if the table schema has evolved Map tableSchema = hoodieHiveClient.getTableSchema(tableName); - SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields, cfg.supportTimestamp); + SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, hiveSyncConfig.partitionFields, hiveSyncConfig.supportTimestamp); if (!schemaDiff.isEmpty()) { LOG.info("Schema difference found for " + tableName); hoodieHiveClient.updateTableDefinition(tableName, schema); // Sync the table properties if the schema has changed - if (cfg.tableProperties != null || cfg.syncAsSparkDataSourceTable) { + if (hiveSyncConfig.tableProperties != null || hiveSyncConfig.syncAsSparkDataSourceTable) { hoodieHiveClient.updateTableProperties(tableName, tableProperties); LOG.info("Sync table properties for " + tableName + ", table properties is: " + tableProperties); } @@ -265,7 +284,7 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea } } - if (cfg.syncComment) { + if (hiveSyncConfig.syncComment) { Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields(); Map newComments = avroSchemaWithoutMetadataFields.getFields() .stream().collect(Collectors.toMap(Schema.Field::name, field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); @@ -290,7 +309,7 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M // The following code refers to the spark code in // https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala GroupType originGroupType = schema.asGroupType(); - List partitionNames = cfg.partitionFields; + List partitionNames = hiveSyncConfig.partitionFields; List partitionCols = new ArrayList<>(); List dataCols = new ArrayList<>(); Map column2Field = new HashMap<>(); @@ -319,8 +338,8 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M Map sparkProperties = new HashMap<>(); sparkProperties.put("spark.sql.sources.provider", "hudi"); - if (!StringUtils.isNullOrEmpty(cfg.sparkVersion)) { - sparkProperties.put("spark.sql.create.version", cfg.sparkVersion); + if (!StringUtils.isNullOrEmpty(hiveSyncConfig.sparkVersion)) { + sparkProperties.put("spark.sql.create.version", hiveSyncConfig.sparkVersion); } // Split the schema string to multi-parts according the schemaLengthThreshold size. String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType); @@ -344,7 +363,7 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M private Map getSparkSerdeProperties(boolean readAsOptimized) { Map sparkSerdeProperties = new HashMap<>(); - sparkSerdeProperties.put("path", cfg.basePath); + sparkSerdeProperties.put("path", hiveSyncConfig.basePath); sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized)); return sparkSerdeProperties; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java index 4bafd854ae318..16c30a16aabc6 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java @@ -18,15 +18,24 @@ package org.apache.hudi.hive.replication; -import com.beust.jcommander.Parameter; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.hive.HiveSyncConfig; +import com.beust.jcommander.Parameter; + public class GlobalHiveSyncConfig extends HiveSyncConfig { @Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters") public String globallyReplicatedTimeStamp; + public GlobalHiveSyncConfig() { + } + + public GlobalHiveSyncConfig(TypedProperties props) { + super(props); + } + public static GlobalHiveSyncConfig copy(GlobalHiveSyncConfig cfg) { - GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(); + GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(cfg.getProps()); newConfig.basePath = cfg.basePath; newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; newConfig.databaseName = cfg.databaseName; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java index 1d225cb840c05..51b2a77ae7433 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java @@ -48,9 +48,9 @@ public void syncHoodieTable() { @Override protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { super.syncHoodieTable(tableName, useRealtimeInputFormat, readAsOptimized); - if (((GlobalHiveSyncConfig)cfg).globallyReplicatedTimeStamp != null) { + if (((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp != null) { hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, - ((GlobalHiveSyncConfig) cfg).globallyReplicatedTimeStamp); + ((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp); } LOG.info("Sync complete for " + tableName); } 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 9fc87fcb456b0..0e23615d5dadd 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 @@ -59,12 +59,15 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.hive.testutils.HiveTestUtil.basePath; import static org.apache.hudi.hive.testutils.HiveTestUtil.ddlExecutor; import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; -import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncConfig; +import static org.apache.hudi.hive.testutils.HiveTestUtil.getHiveConf; +import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; 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.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -79,6 +82,7 @@ private static Iterable syncMode() { return SYNC_MODES; } + // useSchemaFromCommitMetadata, syncMode private static Iterable syncModeAndSchemaFromCommitMetadata() { List opts = new ArrayList<>(); for (Object mode : SYNC_MODES) { @@ -88,6 +92,9 @@ private static Iterable syncModeAndSchemaFromCommitMetadata() { return opts; } + private HiveSyncTool hiveSyncTool; + private HoodieHiveClient hiveClient; + @AfterAll public static void cleanUpClass() { HiveTestUtil.shutdown(); @@ -125,115 +132,100 @@ public void teardown() throws Exception { @ParameterizedTest @MethodSource({"syncModeAndSchemaFromCommitMetadata"}) public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive - // session, then lead to connection retry, we can see there is a exception at log. - hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), - "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); - assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); + assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "No new partition should be added"); - hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "New partition should be added"); // Update partitions - hiveClient.updatePartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Partition count should remain the same"); - hiveClient.updatePartitionsToTable(hiveSyncConfig.tableName, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Partition count should remain the same"); // Alter partitions // Manually change a hive partition location to check if the sync will detect // it and generate a partition update event for it. - ddlExecutor.runSQL("ALTER TABLE `" + hiveSyncConfig.tableName + ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'"); - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty()); - //writtenPartitionsSince.add(newPartition.get(0)); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.UPDATE, partitionEvents.iterator().next().eventType, "The one partition event must of type UPDATE"); - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + // Lets do the sync + reSyncHiveTable(); + // Sync should update the changed partition to correct path - List tablePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List tablePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); assertEquals(6, tablePartitions.size(), "The one partition we wrote should be added to hive"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 100"); } @ParameterizedTest @MethodSource({"syncMode"}) public void testSyncDataBase(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncConfig.databaseName = "database1"; + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), HiveTestUtil.DB_NAME); // while autoCreateDatabase is false and database not exists; - hiveSyncConfig.autoCreateDatabase = false; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); + reinitHiveSyncClient(); // Lets do the sync - assertThrows(Exception.class, () -> { - new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); - }); + assertThrows(Exception.class, (this::reSyncHiveTable)); // while autoCreateDatabase is true and database not exists; - hiveSyncConfig.autoCreateDatabase = true; - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertDoesNotThrow(() -> { - new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); - }); - assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), - "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); + reinitHiveSyncClient(); + assertDoesNotThrow((this::reSyncHiveTable)); + assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), + "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); // while autoCreateDatabase is false and database exists; - hiveSyncConfig.autoCreateDatabase = false; - assertDoesNotThrow(() -> { - new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); - }); - assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), - "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); + reinitHiveSyncClient(); + assertDoesNotThrow((this::reSyncHiveTable)); + assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), + "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); // while autoCreateDatabase is true and database exists; - hiveSyncConfig.autoCreateDatabase = true; - assertDoesNotThrow(() -> { - new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); - }); - assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), - "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); + assertDoesNotThrow((this::reSyncHiveTable)); + assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), + "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); } @ParameterizedTest @@ -241,11 +233,9 @@ public void testSyncDataBase(String syncMode) throws Exception { public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, boolean syncAsDataSourceTable, String syncMode) throws Exception { - HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; Map serdeProperties = new HashMap() { { - put("path", hiveSyncConfig.basePath); + put("path", HiveTestUtil.basePath); } }; @@ -255,20 +245,20 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); - hiveSyncConfig.syncMode = syncMode; - hiveSyncConfig.syncAsSparkDataSourceTable = syncAsDataSourceTable; - hiveSyncConfig.serdeProperties = ConfigUtils.configToString(serdeProperties); - hiveSyncConfig.tableProperties = ConfigUtils.configToString(tableProperties); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); SessionState.start(HiveTestUtil.getHiveConf()); Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); - String dbTableName = hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName; + String dbTableName = HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); List results = new ArrayList<>(); hiveDriver.getResults(results); @@ -279,10 +269,10 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, String sparkTableProperties = getSparkTableProperties(syncAsDataSourceTable, useSchemaFromCommitMetadata); assertEquals( "EXTERNAL\tTRUE\n" - + "last_commit_time_sync\t100\n" - + sparkTableProperties - + "tp_0\tp0\n" - + "tp_1\tp1", tblPropertiesWithoutDdlTime); + + "last_commit_time_sync\t100\n" + + sparkTableProperties + + "tp_0\tp0\n" + + "tp_1\tp1", tblPropertiesWithoutDdlTime); assertTrue(results.get(results.size() - 1).startsWith("transient_lastDdlTime")); results.clear(); @@ -290,7 +280,7 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, hiveDriver.run("SHOW CREATE TABLE " + dbTableName); hiveDriver.getResults(results); String ddl = String.join("\n", results); - assertTrue(ddl.contains("'path'='" + hiveSyncConfig.basePath + "'")); + assertTrue(ddl.contains("'path'='" + HiveTestUtil.basePath + "'")); if (syncAsDataSourceTable) { assertTrue(ddl.contains("'" + ConfigUtils.IS_QUERY_AS_RO_TABLE + "'='false'")); } @@ -299,33 +289,33 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, private String getSparkTableProperties(boolean syncAsDataSourceTable, boolean useSchemaFromCommitMetadata) { if (syncAsDataSourceTable) { if (useSchemaFromCommitMetadata) { - return "spark.sql.sources.provider\thudi\n" - + "spark.sql.sources.schema.numPartCols\t1\n" - + "spark.sql.sources.schema.numParts\t1\n" - + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":" - + "[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"name\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"favorite_number\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" - + "spark.sql.sources.schema.partCol.0\tdatestr\n"; + return "spark.sql.sources.provider\thudi\n" + + "spark.sql.sources.schema.numPartCols\t1\n" + + "spark.sql.sources.schema.numParts\t1\n" + + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":" + + "[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"name\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"favorite_number\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" + + "spark.sql.sources.schema.partCol.0\tdatestr\n"; } else { return "spark.sql.sources.provider\thudi\n" - + "spark.sql.sources.schema.numPartCols\t1\n" - + "spark.sql.sources.schema.numParts\t1\n" - + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":" - + "\"string\",\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_number\",\"type\":\"integer\"," - + "\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false," - + "\"metadata\":{}}]}\n" - + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" - + "spark.sql.sources.schema.partCol.0\tdatestr\n"; + + "spark.sql.sources.schema.numPartCols\t1\n" + + "spark.sql.sources.schema.numParts\t1\n" + + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":" + + "\"string\",\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_number\",\"type\":\"integer\"," + + "\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false," + + "\"metadata\":{}}]}\n" + + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" + + "spark.sql.sources.schema.partCol.0\tdatestr\n"; } } else { - return ""; + return ""; } } @@ -334,11 +324,9 @@ private String getSparkTableProperties(boolean syncAsDataSourceTable, boolean us public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, boolean syncAsDataSourceTable, String syncMode) throws Exception { - HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; Map serdeProperties = new HashMap() { { - put("path", hiveSyncConfig.basePath); + put("path", HiveTestUtil.basePath); } }; @@ -348,20 +336,21 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; - hiveSyncConfig.syncAsSparkDataSourceTable = syncAsDataSourceTable; - hiveSyncConfig.syncMode = syncMode; - hiveSyncConfig.serdeProperties = ConfigUtils.configToString(serdeProperties); - hiveSyncConfig.tableProperties = ConfigUtils.configToString(tableProperties); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); + String instantTime = "100"; String deltaCommitTime = "101"; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); - String roTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; - String rtTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String roTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; + String rtTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; String[] tableNames = new String[] {roTableName, rtTableName}; String[] readAsOptimizedResults = new String[] {"true", "false"}; @@ -370,8 +359,8 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); String sparkTableProperties = getSparkTableProperties(syncAsDataSourceTable, useSchemaFromCommitMetadata); - for (int i = 0;i < 2; i++) { - String dbTableName = hiveSyncConfig.databaseName + "." + tableNames[i]; + for (int i = 0; i < 2; i++) { + String dbTableName = HiveTestUtil.DB_NAME + "." + tableNames[i]; String readAsOptimized = readAsOptimizedResults[i]; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); @@ -382,10 +371,10 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, results.subList(0, results.size() - 1)); assertEquals( "EXTERNAL\tTRUE\n" - + "last_commit_time_sync\t101\n" - + sparkTableProperties - + "tp_0\tp0\n" - + "tp_1\tp1", tblPropertiesWithoutDdlTime); + + "last_commit_time_sync\t101\n" + + sparkTableProperties + + "tp_0\tp0\n" + + "tp_1\tp1", tblPropertiesWithoutDdlTime); assertTrue(results.get(results.size() - 1).startsWith("transient_lastDdlTime")); results.clear(); @@ -393,7 +382,7 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, hiveDriver.run("SHOW CREATE TABLE " + dbTableName); hiveDriver.getResults(results); String ddl = String.join("\n", results); - assertTrue(ddl.contains("'path'='" + hiveSyncConfig.basePath + "'")); + assertTrue(ddl.contains("'path'='" + HiveTestUtil.basePath + "'")); assertTrue(ddl.toLowerCase().contains("create external table")); if (syncAsDataSourceTable) { assertTrue(ddl.contains("'" + ConfigUtils.IS_QUERY_AS_RO_TABLE + "'='" + readAsOptimized + "'")); @@ -406,19 +395,18 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, boolean isManagedTable, String syncMode) throws Exception { - HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key(), String.valueOf(isManagedTable)); - hiveSyncConfig.syncMode = syncMode; - hiveSyncConfig.createManagedTable = isManagedTable; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); SessionState.start(HiveTestUtil.getHiveConf()); Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); - String dbTableName = hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName; + String dbTableName = HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); List results = new ArrayList<>(); @@ -435,37 +423,29 @@ public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, @ParameterizedTest @MethodSource("syncMode") public void testSyncWithSchema(String syncMode) throws Exception { - - hiveSyncConfig.syncMode = syncMode; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/complex.schema.avsc"); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + reinitHiveSyncClient(); + reSyncHiveTable(); + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); } @ParameterizedTest @MethodSource("syncMode") public void testSyncIncremental(String syncMode) throws Exception { - - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + reinitHiveSyncClient(); + reSyncHiveTable(); + assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Now lets create more partitions and these are the only ones which needs to be synced @@ -474,37 +454,32 @@ public void testSyncIncremental(String syncMode) throws Exception { HiveTestUtil.addCOWPartitions(1, true, true, dateTime, commitTime2); // Lets do the sync - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reSyncHiveTable(); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1)); assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit"); - List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD"); - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "The one partition we wrote should be added to hive"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 101"); } @ParameterizedTest @MethodSource("syncMode") public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); - int fields = hiveClient.getTableSchema(hiveSyncConfig.tableName).size(); + int fields = hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -512,33 +487,30 @@ public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Excep HiveTestUtil.addCOWPartitions(1, false, true, dateTime, commitTime2); // Lets do the sync - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - - assertEquals(fields + 3, hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), + reinitHiveSyncClient(); + reSyncHiveTable(); + assertEquals(fields + 3, hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema has evolved and should not be 3 more field"); - assertEquals("BIGINT", hiveClient.getTableSchema(hiveSyncConfig.tableName).get("favorite_number"), + assertEquals("BIGINT", hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("favorite_number"), "Hive Schema has evolved - Field favorite_number has evolved from int to long"); - assertTrue(hiveClient.getTableSchema(hiveSyncConfig.tableName).containsKey("favorite_movie"), + assertTrue(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).containsKey("favorite_movie"), "Hive Schema has evolved - Field favorite_movie was added"); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "The one partition we wrote should be added to hive"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 101"); } @ParameterizedTest @MethodSource("syncMode") public void testUpdateTableComments(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test.avsc"); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); + reSyncHiveTable(); Map> alterCommentSchema = new HashMap<>(); //generate commented schema field @@ -554,9 +526,9 @@ public void testUpdateTableComments(String syncMode) throws Exception { } } - ddlExecutor.updateTableComments(hiveSyncConfig.tableName,alterCommentSchema); + ddlExecutor.updateTableComments(HiveTestUtil.TABLE_NAME, alterCommentSchema); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -569,16 +541,14 @@ public void testUpdateTableComments(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncWithCommentedSchema(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - hiveSyncConfig.syncComment = false; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "false"); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test-doced.avsc"); - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + reinitHiveSyncClient(); + reSyncHiveTable(); + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -587,10 +557,10 @@ public void testSyncWithCommentedSchema(String syncMode) throws Exception { } assertEquals(0, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); - hiveSyncConfig.syncComment = true; - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "true"); + reinitHiveSyncClient(); + reSyncHiveTable(); + fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -603,31 +573,29 @@ public void testSyncWithCommentedSchema(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; String deltaCommitTime = "101"; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - String roTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; - HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(roTableName), "Table " + hiveSyncConfig.tableName + " should not exist initially"); + String roTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(roTableName), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reSyncHiveTable(); assertTrue(hiveClient.doesTableExist(roTableName), "Table " + roTableName + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "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(), - SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the table schema + partition field"); } @@ -645,19 +613,18 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync HiveTestUtil.addMORPartitions(1, true, false, useSchemaFromCommitMetadata, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); + reSyncHiveTable(); if (useSchemaFromCommitMetadata) { assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "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(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition @@ -670,42 +637,38 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; String deltaCommitTime = "101"; - String snapshotTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - HoodieHiveClient hiveClientRT = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - - assertFalse(hiveClientRT.doesTableExist(snapshotTableName), - "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(snapshotTableName), + "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reSyncHiveTable(); - assertTrue(hiveClientRT.doesTableExist(snapshotTableName), - "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertTrue(hiveClient.doesTableExist(snapshotTableName), + "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "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(), - SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the table schema + partition field"); } - assertEquals(5, hiveClientRT.scanTablePartitions(snapshotTableName).size(), + assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(deltaCommitTime, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(deltaCommitTime, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Now lets create more partitions and these are the only ones which needs to be synced @@ -716,57 +679,52 @@ public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String sy HiveTestUtil.addCOWPartitions(1, true, useSchemaFromCommitMetadata, dateTime, commitTime2); HiveTestUtil.addMORPartitions(1, true, false, useSchemaFromCommitMetadata, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - hiveClientRT = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); + reSyncHiveTable(); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "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(hiveClientRT.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition - assertEquals(6, hiveClientRT.scanTablePartitions(snapshotTableName).size(), + assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), "The 2 partitions we wrote should be added to hive"); - assertEquals(deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); } @ParameterizedTest @MethodSource("syncMode") public void testMultiPartitionKeySync(String syncMode) throws Exception { - - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(HiveTestUtil.hiveSyncConfig); - hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName(); - hiveSyncConfig.tableName = "multi_part_key"; - hiveSyncConfig.partitionFields = Arrays.asList("year", "month", "day"); - HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day"); - HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); + + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // HoodieHiveClient had a bug where partition vals were sorted @@ -775,41 +733,36 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { String commitTime2 = "101"; HiveTestUtil.addCOWPartition("2010/01/02", true, true, commitTime2); - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); 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 hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD"); - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - + reSyncHiveTable(); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced 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(), fileSystem); + HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - - assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), + reinitHiveSyncClient(); + reSyncHiveTable(); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(7, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(7, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).size()); } @@ -817,49 +770,43 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartitionKeySync(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive - // session, then lead to connection retry, we can see there is a exception at log. - hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), - "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); - assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "No new partition should be added"); - hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, newPartition); - assertEquals(2, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); + assertEquals(2, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "New partition should be added"); - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reSyncHiveTable(); // Drop 1 partition. - ddlExecutor.runSQL("ALTER TABLE `" + hiveSyncConfig.tableName + ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + "` DROP PARTITION (`datestr`='2050-01-01')"); - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); assertEquals(1, hivePartitions.size(), "Table should have 1 partition because of the drop 1 partition"); } @@ -867,42 +814,35 @@ public void testDropPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartition(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive - // session, then lead to connection retry, we can see there is a exception at log. - hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), - "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - List partitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List partitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); assertEquals(1, partitions.size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); String partitiontoDelete = partitions.get(0).getValues().get(0).replace("-","/"); // create a replace commit to delete current partitions+ HiveTestUtil.createReplaceCommit("101", partitiontoDelete, WriteOperationType.DELETE_PARTITION, true, true); - // sync drop partitins - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + // sync drop partitions + reinitHiveSyncClient(); + reSyncHiveTable(); - hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); + List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); assertEquals(0, hivePartitions.size(), "Table should have 0 partition because of the drop the only one partition"); } @@ -910,62 +850,53 @@ public void testDropPartition(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testNonPartitionedSync(String syncMode) throws Exception { - - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - - HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(HiveTestUtil.hiveSyncConfig); // Set partition value extractor to NonPartitionedExtractor - hiveSyncConfig.partitionValueExtractorClass = NonPartitionedExtractor.class.getCanonicalName(); - hiveSyncConfig.tableName = "non_partitioned"; - hiveSyncConfig.partitionFields = Arrays.asList("year", "month", "day"); - HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); - - HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year, month, day"); + + HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); + + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), + reSyncHiveTable(); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size(), "Hive Schema should match the table schema,ignoring the partition fields"); - assertEquals(0, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), + assertEquals(0, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table should not have partitions because of the NonPartitionedExtractor"); } @ParameterizedTest @MethodSource("syncMode") public void testReadSchemaForMOR(String syncMode) throws Exception { - - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; - String snapshotTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; HiveTestUtil.createMORTable(commitTime, "", 5, false, true); - HoodieHiveClient hiveClientRT = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); - assertFalse(hiveClientRT.doesTableExist(snapshotTableName), "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertFalse(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should not exist initially"); // Lets do the sync - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); + reSyncHiveTable(); - assertTrue(hiveClientRT.doesTableExist(snapshotTableName), "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertTrue(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should exist after sync completes"); // Schema being read from compacted base files - assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); + assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -974,84 +905,78 @@ public void testReadSchemaForMOR(String syncMode) throws Exception { HiveTestUtil.addMORPartitions(1, true, false, true, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - tool.syncHoodieTable(); - hiveClientRT = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + reinitHiveSyncClient(); + reSyncHiveTable(); - // Schema being read from the log files - assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + // Schema being read from the log filesTestHiveSyncTool + assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); // Sync should add the one partition - assertEquals(6, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); - assertEquals(deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); + assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); } @Test public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxException, HiveException, MetaException { - hiveSyncConfig.useJdbc = true; - HiveTestUtil.hiveSyncConfig.useJdbc = true; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, false); - HoodieHiveClient hiveClient = - new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); - // Lets do the sync - - HiveSyncConfig syncToolConfig = HiveSyncConfig.copy(hiveSyncConfig); - syncToolConfig.ignoreExceptions = true; - syncToolConfig.jdbcUrl = HiveTestUtil.hiveSyncConfig.jdbcUrl - .replace(String.valueOf(HiveTestUtil.hiveTestService.getHiveServerPort()), String.valueOf(NetworkTestUtils.nextFreePort())); - HiveSyncTool tool = new HiveSyncTool(syncToolConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + HoodieHiveClient prevHiveClient = hiveClient; + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); - assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), - "Table " + hiveSyncConfig.tableName + " should not exist initially"); + // Lets do the sync + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key(), "true"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key()) + .replace(String.valueOf(HiveTestUtil.hiveTestService.getHiveServerPort()), String.valueOf(NetworkTestUtils.nextFreePort()))); + reinitHiveSyncClient(); + reSyncHiveTable(); + + assertNull(hiveClient); + assertFalse(prevHiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), + "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); } private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyCommitTime) throws Exception { - assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.hiveSyncConfig.tableName).size(),"Table partitions should match the number of partitions we wrote"); + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(emptyCommitTime, - hiveClient.getLastCommitTimeSynced(HiveTestUtil.hiveSyncConfig.tableName).get(),"The last commit that was synced should be updated in the TBLPROPERTIES"); + hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // make sure correct schema is picked Schema schema = SchemaTestUtil.getSimpleSchema(); for (Field field : schema.getFields()) { assertEquals(field.schema().getType().getName(), - hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get(field.name()).toLowerCase(), + hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get(field.name()).toLowerCase(), String.format("Hive Schema Field %s was added", field)); } assertEquals("string", - hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); + hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); assertEquals(schema.getFields().size() + 1 + HoodieRecord.HOODIE_META_COLUMNS.size(), - hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(),"Hive Schema fields size"); + hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema fields size"); } @ParameterizedTest @MethodSource("syncMode") public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); // create empty commit final String emptyCommitTime = "200"; HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + reinitHiveSyncClient(); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); - HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); verifyOldParquetFileTest(hiveClient, emptyCommitTime); } @@ -1059,8 +984,7 @@ public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) th @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -1072,17 +996,15 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM // create empty commit final String emptyCommitTime = "200"; - HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime, hiveSyncConfig.basePath); + HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime, basePath); - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + reinitHiveSyncClient(); assertFalse( - hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); - - HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf(), fileSystem); // now delete the evolved commit instant - Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + hiveClient.getActiveTimeline().getInstants() .filter(inst -> inst.getTimestamp().equals(commitTime2)) .findFirst().get().getFileName()); @@ -1095,15 +1017,13 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM } // table should not be synced yet - assertFalse( - hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist at all"); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist at all"); } @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTable(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -1111,13 +1031,11 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa final String emptyCommitTime = "200"; HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); //HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime); - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + reinitHiveSyncClient(); assertFalse( - hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); - HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reSyncHiveTable(); verifyOldParquetFileTest(hiveClient, emptyCommitTime); @@ -1128,18 +1046,19 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa //HiveTestUtil.createCommitFileWithSchema(commitMetadata, "400", false); // create another empty commit //HiveTestUtil.createCommitFile(commitMetadata, "400"); // create another empty commit - tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - HoodieHiveClient hiveClientLatest = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + reinitHiveSyncClient(); // now delete the evolved commit instant - Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + hiveClientLatest.getActiveTimeline().getInstants() - .filter(inst -> inst.getTimestamp().equals(commitTime2)) - .findFirst().get().getFileName()); + Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + hiveClient.getActiveTimeline().getInstants() + .filter(inst -> inst.getTimestamp().equals(commitTime2)) + .findFirst().get().getFileName()); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); try { - tool.syncHoodieTable(); + reSyncHiveTable(); } catch (RuntimeException e) { // we expect the table sync to fail + } finally { + reinitHiveSyncClient(); } // old sync values should be left intact @@ -1149,15 +1068,13 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa @ParameterizedTest @MethodSource("syncMode") public void testTypeConverter(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); HiveTestUtil.createCOWTable("100", 5, true); // create database. - ddlExecutor.runSQL("create database " + hiveSyncConfig.databaseName); - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - String tableName = HiveTestUtil.hiveSyncConfig.tableName; - String tableAbsoluteName = String.format(" `%s.%s` ", HiveTestUtil.hiveSyncConfig.databaseName, tableName); + ddlExecutor.runSQL("create database " + HiveTestUtil.DB_NAME); + reinitHiveSyncClient(); + String tableName = HiveTestUtil.TABLE_NAME; + String tableAbsoluteName = String.format(" `%s.%s` ", HiveTestUtil.DB_NAME, tableName); String dropTableSql = String.format("DROP TABLE IF EXISTS %s ", tableAbsoluteName); String createTableSqlPrefix = String.format("CREATE TABLE IF NOT EXISTS %s ", tableAbsoluteName); String errorMsg = "An error occurred in decimal type converting."; @@ -1191,31 +1108,40 @@ public void testTypeConverter(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncWithoutDiffs(String syncMode) throws Exception { - hiveSyncConfig.syncMode = syncMode; - hiveSyncConfig.isConditionalSync = true; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; - String tableName = HiveTestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String tableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_CONDITIONAL_SYNC.key(), "true"); String commitTime0 = "100"; String commitTime1 = "101"; String commitTime2 = "102"; HiveTestUtil.createMORTable(commitTime0, commitTime1, 2, true, true); - HoodieHiveClient hiveClient = - new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - - HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); + reinitHiveSyncClient(); + reSyncHiveTable(); assertTrue(hiveClient.doesTableExist(tableName)); assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); HiveTestUtil.addMORPartitions(0, true, true, true, ZonedDateTime.now().plusDays(2), commitTime1, commitTime2); - tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - tool.syncHoodieTable(); - hiveClient = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + reSyncHiveTable(); assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); } + private void reSyncHiveTable() { + hiveSyncTool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + reinitHiveSyncClient(); + } + + private void reinitHiveSyncClient() { + hiveSyncTool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); + hiveClient = hiveSyncTool.hoodieHiveClient; + } + + private int getPartitionFieldSize() { + return hiveSyncProps.getString(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key()).split(",").length; + } } 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 e66bb7c914645..3cdbe0d8bb757 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 @@ -22,6 +22,7 @@ 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.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieBaseFile; @@ -75,7 +76,6 @@ import java.time.format.DateTimeFormatter; import java.time.temporal.ChronoUnit; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -89,16 +89,21 @@ @SuppressWarnings("SameParameterValue") public class HiveTestUtil { + public static final String DB_NAME = "testdb"; + public static final String TABLE_NAME = "test1"; + public static String basePath; + public static TypedProperties hiveSyncProps; + public static HiveTestService hiveTestService; + public static FileSystem fileSystem; + public static QueryBasedDDLExecutor ddlExecutor; + private static ZooKeeperServer zkServer; private static HiveServer2 hiveServer; - public static HiveTestService hiveTestService; private static ZookeeperTestService zkService; private static Configuration configuration; - public static HiveSyncConfig hiveSyncConfig; + private static HiveSyncConfig hiveSyncConfig; private static DateTimeFormatter dtfOut; - public static FileSystem fileSystem; private static Set createdTablesSet = new HashSet<>(); - public static QueryBasedDDLExecutor ddlExecutor; public static void setUp() throws IOException, InterruptedException, HiveException, MetaException { configuration = new Configuration(); @@ -112,16 +117,21 @@ public static void setUp() throws IOException, InterruptedException, HiveExcepti } fileSystem = FileSystem.get(configuration); - hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.jdbcUrl = hiveTestService.getJdbcHive2Url(); - hiveSyncConfig.hiveUser = ""; - hiveSyncConfig.hivePass = ""; - hiveSyncConfig.databaseName = "testdb"; - hiveSyncConfig.tableName = "test1"; - hiveSyncConfig.basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString(); - hiveSyncConfig.assumeDatePartitioning = true; - hiveSyncConfig.usePreApacheInputFormat = false; - hiveSyncConfig.partitionFields = Collections.singletonList("datestr"); + basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString(); + + hiveSyncProps = new TypedProperties(); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveTestService.getJdbcHive2Url()); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USER.key(), ""); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_PASS.key(), ""); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), DB_NAME); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), TABLE_NAME); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_BASE_PATH.key(), basePath); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM.key(), "3"); + + hiveSyncConfig = new HiveSyncConfig(hiveSyncProps); dtfOut = DateTimeFormatter.ofPattern("yyyy/MM/dd"); ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig, fileSystem, getHiveConf()); @@ -138,18 +148,18 @@ public static void clearIncrementalPullSetup(String path1, String path2) throws } public static void clear() throws IOException, HiveException, MetaException { - fileSystem.delete(new Path(hiveSyncConfig.basePath), true); + fileSystem.delete(new Path(basePath), true); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) + .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .initTable(configuration, basePath); for (String tableName : createdTablesSet) { ddlExecutor.runSQL("drop table if exists " + tableName); } createdTablesSet.clear(); - ddlExecutor.runSQL("drop database if exists " + hiveSyncConfig.databaseName + " cascade"); + ddlExecutor.runSQL("drop database if exists " + DB_NAME + " cascade"); } public static HiveConf getHiveConf() { @@ -189,7 +199,7 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo public static void createCOWTable(String instantTime, int numberOfPartitions, boolean useSchemaFromCommitMetadata) throws IOException, URISyntaxException { - createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, hiveSyncConfig.basePath, hiveSyncConfig.databaseName, hiveSyncConfig.tableName); + createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, basePath, DB_NAME, TABLE_NAME); } public static void createReplaceCommit(String instantTime, String partitions, WriteOperationType type, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata) @@ -205,13 +215,13 @@ public static void createReplaceCommit(String instantTime, String partitions, Wr public static void createCOWTableWithSchema(String instantTime, String schemaFileName) throws IOException, URISyntaxException { - Path path = new Path(hiveSyncConfig.basePath); - FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); + Path path = new Path(basePath); + FileIOUtils.deleteDirectory(new File(basePath)); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) + .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .initTable(configuration, basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); @@ -219,7 +229,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); String partitionPath = dateTime.format(dtfOut); - Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath); + Path partPath = new Path(basePath + "/" + partitionPath); fileSystem.makeQualified(partPath); fileSystem.mkdirs(partPath); List writeStats = new ArrayList<>(); @@ -233,30 +243,30 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil writeStats.add(writeStat); writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s)); commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schema.toString()); - createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); - createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath); + createdTablesSet.add(DB_NAME + "." + TABLE_NAME); + createCommitFile(commitMetadata, instantTime, basePath); } public static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions, boolean createDeltaCommit, boolean useSchemaFromCommitMetadata) throws IOException, URISyntaxException, InterruptedException { - Path path = new Path(hiveSyncConfig.basePath); - FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); + Path path = new Path(basePath); + FileIOUtils.deleteDirectory(new File(basePath)); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) - .setTableName(hiveSyncConfig.tableName) + .setTableName(TABLE_NAME) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .initTable(configuration, basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); ZonedDateTime dateTime = ZonedDateTime.now(); HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, - useSchemaFromCommitMetadata, dateTime, commitTime, hiveSyncConfig.basePath); + useSchemaFromCommitMetadata, dateTime, commitTime, basePath); createdTablesSet - .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); + .add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); createdTablesSet - .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); + .add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -274,26 +284,26 @@ public static void createMORTable(String commitTime, String deltaCommitTime, int public static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata, ZonedDateTime startFrom, String instantTime) throws IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = - createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime, hiveSyncConfig.basePath); - createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); - createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath); + createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime, basePath); + createdTablesSet.add(DB_NAME + "." + TABLE_NAME); + createCommitFile(commitMetadata, instantTime, basePath); } 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, hiveSyncConfig.basePath); + createdTablesSet.add(DB_NAME + "." + TABLE_NAME); + createCommitFile(commitMetadata, instantTime, basePath); } public static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean isLogSchemaSimple, boolean useSchemaFromCommitMetadata, ZonedDateTime startFrom, String instantTime, String deltaCommitTime) throws IOException, URISyntaxException, InterruptedException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, - useSchemaFromCommitMetadata, startFrom, instantTime, hiveSyncConfig.basePath); - createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); - createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); + useSchemaFromCommitMetadata, startFrom, instantTime, basePath); + createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); + createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -346,7 +356,7 @@ private static HoodieCommitMetadata createPartitions(int numberOfPartitions, boo 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); + Path partPath = new Path(basePath + "/" + partitionPath); fileSystem.makeQualified(partPath); fileSystem.mkdirs(partPath); List writeStats = createTestData(partPath, isParquetSchemaSimple, instantTime); @@ -471,7 +481,7 @@ public static void createCommitFile(HoodieCommitMetadata commitMetadata, String public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeReplaceFileName(instantTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); @@ -480,13 +490,13 @@ public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, public static void createCommitFileWithSchema(HoodieCommitMetadata commitMetadata, String instantTime, boolean isSimpleSchema) throws IOException { addSchemaToCommitMetadata(commitMetadata, isSimpleSchema, true); - createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath); + createCommitFile(commitMetadata, instantTime, basePath); } private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(instantTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); @@ -496,7 +506,7 @@ private static void createCompactionCommitFile(HoodieCommitMetadata commitMetada private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime) throws IOException { byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(deltaCommitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index 1f1abb4f177f1..19a83e757c0d4 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -45,6 +45,65 @@ org.apache.hadoop hadoop-common + + com.beust + jcommander + + + + + org.apache.logging.log4j + log4j-core + test + + + + 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 + diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java index 6621468ee2e25..680b4a17ef5d9 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java @@ -17,17 +17,31 @@ package org.apache.hudi.sync.common; +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import java.util.Properties; +/** + * Base class to sync Hudi meta data with Metastores to make + * Hudi table queryable through external systems. + */ public abstract class AbstractSyncTool { - protected Properties props; - protected FileSystem fileSystem; + protected final Configuration conf; + protected final FileSystem fs; + protected TypedProperties props; - public AbstractSyncTool(Properties props, FileSystem fileSystem) { + public AbstractSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { this.props = props; - this.fileSystem = fileSystem; + this.conf = conf; + this.fs = fs; + } + + @Deprecated + public AbstractSyncTool(Properties props, FileSystem fileSystem) { + this(new TypedProperties(props), fileSystem.getConf(), fileSystem); } public abstract void syncHoodieTable(); diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java new file mode 100644 index 0000000000000..dc2b21ba4504f --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java @@ -0,0 +1,188 @@ +/* + * 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.sync.common; + +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; + +import com.beust.jcommander.Parameter; + +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +/** + * Configs needed to sync data into external meta stores, catalogs, etc. + */ +public class HoodieSyncConfig extends HoodieConfig { + + @Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true) + public String databaseName; + + @Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true) + public String tableName; + + @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) + public String basePath; + + @Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)") + public String baseFileFormat; + + @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by") + public List partitionFields; + + @Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor " + + "to extract the partition values from HDFS path") + public String partitionValueExtractorClass; + + @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" + + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") + public Boolean assumeDatePartitioning; + + @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") + public Boolean decodePartition; + + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata; + + @Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.") + public Boolean isConditionalSync; + + @Parameter(names = {"--spark-version"}, description = "The spark version") + public String sparkVersion; + + public static final ConfigProperty META_SYNC_BASE_PATH = ConfigProperty + .key("hoodie.datasource.meta.sync.base.path") + .defaultValue("") + .withDocumentation("Base path of the hoodie table to sync"); + + public static final ConfigProperty META_SYNC_ENABLED = ConfigProperty + .key("hoodie.datasource.meta.sync.enable") + .defaultValue("false") + .withDocumentation("Enable Syncing the Hudi Table with an external meta store or data catalog."); + + // ToDo change the prefix of the following configs from hive_sync to meta_sync + public static final ConfigProperty META_SYNC_DATABASE_NAME = ConfigProperty + .key("hoodie.datasource.hive_sync.database") + .defaultValue("default") + .withDocumentation("The name of the destination database that we should sync the hudi table to."); + + // If the table name for the metastore destination is not provided, pick it up from write or table configs. + public static final Function> TABLE_NAME_INFERENCE_FUNCTION = cfg -> { + if (cfg.contains(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)) { + return Option.of(cfg.getString(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)); + } else if (cfg.contains(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)) { + return Option.of(cfg.getString(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)); + } else { + return Option.empty(); + } + }; + public static final ConfigProperty META_SYNC_TABLE_NAME = ConfigProperty + .key("hoodie.datasource.hive_sync.table") + .defaultValue("unknown") + .withInferFunction(TABLE_NAME_INFERENCE_FUNCTION) + .withDocumentation("The name of the destination table that we should sync the hudi table to."); + + public static final ConfigProperty META_SYNC_BASE_FILE_FORMAT = ConfigProperty + .key("hoodie.datasource.hive_sync.base_file_format") + .defaultValue("PARQUET") + .withDocumentation("Base file format for the sync."); + + // If partition fields are not explicitly provided, obtain from the KeyGeneration Configs + public static final Function> PARTITION_FIELDS_INFERENCE_FUNCTION = cfg -> { + if (cfg.contains(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)) { + return Option.of(cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)); + } else { + return Option.empty(); + } + }; + public static final ConfigProperty META_SYNC_PARTITION_FIELDS = ConfigProperty + .key("hoodie.datasource.hive_sync.partition_fields") + .defaultValue("") + .withInferFunction(PARTITION_FIELDS_INFERENCE_FUNCTION) + .withDocumentation("Field in the table to use for determining hive partition columns."); + + // If partition value extraction class is not explicitly provided, configure based on the partition fields. + public static final Function> PARTITION_EXTRACTOR_CLASS_FUNCTION = cfg -> { + if (!cfg.contains(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)) { + return Option.of("org.apache.hudi.hive.NonPartitionedExtractor"); + } else { + int numOfPartFields = cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME).split(",").length; + if (numOfPartFields == 1 + && cfg.contains(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE) + && cfg.getString(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE).equals("true")) { + return Option.of("org.apache.hudi.hive.HiveStylePartitionValueExtractor"); + } else { + return Option.of("org.apache.hudi.hive.MultiPartKeysValueExtractor"); + } + } + }; + public static final ConfigProperty META_SYNC_PARTITION_EXTRACTOR_CLASS = ConfigProperty + .key("hoodie.datasource.hive_sync.partition_extractor_class") + .defaultValue("org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor") + .withInferFunction(PARTITION_EXTRACTOR_CLASS_FUNCTION) + .withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, " + + "default 'SlashEncodedDayPartitionValueExtractor'."); + + public static final ConfigProperty META_SYNC_ASSUME_DATE_PARTITION = ConfigProperty + .key("hoodie.datasource.hive_sync.assume_date_partitioning") + .defaultValue("false") + .withDocumentation("Assume partitioning is yyyy/mm/dd"); + + public static final ConfigProperty META_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty + .key("hoodie.meta.sync.metadata_file_listing") + .defaultValue(HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS) + .withDocumentation("Enable the internal metadata table for file listing for syncing with metastores"); + + public static final ConfigProperty META_SYNC_CONDITIONAL_SYNC = ConfigProperty + .key("hoodie.datasource.meta_sync.condition.sync") + .defaultValue("false") + .withDocumentation("If true, only sync on conditions like schema change or partition change."); + + public static final ConfigProperty META_SYNC_SPARK_VERSION = ConfigProperty + .key("hoodie.meta_sync.spark.version") + .defaultValue("") + .withDocumentation("The spark version used when syncing with a metastore."); + + public HoodieSyncConfig(TypedProperties props) { + super(props); + setDefaults(); + + this.basePath = getStringOrDefault(META_SYNC_BASE_PATH); + this.databaseName = getStringOrDefault(META_SYNC_DATABASE_NAME); + this.tableName = getStringOrDefault(META_SYNC_TABLE_NAME); + this.baseFileFormat = getStringOrDefault(META_SYNC_BASE_FILE_FORMAT); + this.partitionFields = props.getStringList(META_SYNC_PARTITION_FIELDS.key(), ",", Collections.emptyList()); + this.partitionValueExtractorClass = getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS); + this.assumeDatePartitioning = getBooleanOrDefault(META_SYNC_ASSUME_DATE_PARTITION); + this.decodePartition = getBooleanOrDefault(KeyGeneratorOptions.URL_ENCODE_PARTITIONING); + this.useFileListingFromMetadata = getBooleanOrDefault(META_SYNC_USE_FILE_LISTING_FROM_METADATA); + this.isConditionalSync = getBooleanOrDefault(META_SYNC_CONDITIONAL_SYNC); + this.sparkVersion = getStringOrDefault(META_SYNC_SPARK_VERSION); + } + + protected void setDefaults() { + this.setDefaultValue(META_SYNC_TABLE_NAME); + } +} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java new file mode 100644 index 0000000000000..f40e53b80f871 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java @@ -0,0 +1,72 @@ +package org.apache.hudi.sync.common.util; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Properties; + +/** + * Helper class for syncing Hudi commit data with external metastores. + */ +public class SyncUtilHelpers { + private static final Logger LOG = LogManager.getLogger(SyncUtilHelpers.class); + + /** + * Create an instance of an implementation of {@link AbstractSyncTool} that will sync all the relevant meta information + * with an external metastore such as Hive etc. to ensure Hoodie tables can be queried or read via external systems. + * + * @param metaSyncFQCN The class that implements the sync of the metadata. + * @param props property map. + * @param hadoopConfig Hadoop confs. + * @param fs Filesystem used. + * @param targetBasePath The target base path that contains the hoodie table. + * @param baseFileFormat The file format used by the hoodie table (defaults to PARQUET). + */ + public static void runHoodieMetaSync(String metaSyncFQCN, + TypedProperties props, + Configuration hadoopConfig, + FileSystem fs, + String targetBasePath, + String baseFileFormat) { + try { + instantiateMetaSyncTool(metaSyncFQCN, props, hadoopConfig, fs, targetBasePath, baseFileFormat).syncHoodieTable(); + } catch (Throwable e) { + throw new HoodieException("Could not sync using the meta sync class " + metaSyncFQCN, e); + } + } + + static AbstractSyncTool instantiateMetaSyncTool(String metaSyncFQCN, + TypedProperties props, + Configuration hadoopConfig, + FileSystem fs, + String targetBasePath, + String baseFileFormat) { + TypedProperties properties = new TypedProperties(); + properties.putAll(props); + properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), targetBasePath); + properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), baseFileFormat); + + if (ReflectionUtils.hasConstructor(metaSyncFQCN, + new Class[] {TypedProperties.class, Configuration.class, FileSystem.class})) { + return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + new Class[] {TypedProperties.class, Configuration.class, FileSystem.class}, + properties, hadoopConfig, fs)); + } else { + LOG.warn("Falling back to deprecated constructor for class: " + metaSyncFQCN); + try { + return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + new Class[] {Properties.class, FileSystem.class}, properties, fs)); + } catch (Throwable t) { + throw new HoodieException("Could not load meta sync class " + metaSyncFQCN, t); + } + } + } +} diff --git a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java new file mode 100644 index 0000000000000..dc9dee8b42ea4 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java @@ -0,0 +1,124 @@ +/* + * 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.sync.common.util; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.sync.common.AbstractSyncTool; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestSyncUtilHelpers { + private static final String BASE_PATH = "/tmp/test"; + private static final String BASE_FORMAT = "PARQUET"; + + private Configuration hadoopConf; + private FileSystem fileSystem; + + @BeforeEach + public void setUp() throws IOException { + fileSystem = FSUtils.getFs(BASE_PATH, new Configuration()); + hadoopConf = fileSystem.getConf(); + } + + @Test + public void testCreateValidSyncClass() { + AbstractSyncTool metaSyncTool = SyncUtilHelpers.instantiateMetaSyncTool( + ValidMetaSyncClass.class.getName(), + new TypedProperties(), + hadoopConf, + fileSystem, + BASE_PATH, + BASE_FORMAT + ); + assertTrue(metaSyncTool instanceof ValidMetaSyncClass); + } + + /** + * Ensure it still works for the deprecated constructor of {@link AbstractSyncTool} + * as we implemented the fallback. + */ + @Test + public void testCreateDeprecatedSyncClass() { + Properties properties = new Properties(); + AbstractSyncTool deprecatedMetaSyncClass = SyncUtilHelpers.instantiateMetaSyncTool( + DeprecatedMetaSyncClass.class.getName(), + new TypedProperties(properties), + hadoopConf, + fileSystem, + BASE_PATH, + BASE_FORMAT + ); + assertTrue(deprecatedMetaSyncClass instanceof DeprecatedMetaSyncClass); + } + + @Test + public void testCreateInvalidSyncClass() { + Exception exception = assertThrows(HoodieException.class, () -> { + SyncUtilHelpers.instantiateMetaSyncTool( + InvalidSyncClass.class.getName(), + new TypedProperties(), + hadoopConf, + fileSystem, + BASE_PATH, + BASE_FORMAT + ); + }); + + String expectedMessage = "Could not load meta sync class " + InvalidSyncClass.class.getName(); + assertTrue(exception.getMessage().contains(expectedMessage)); + + } + + public static class ValidMetaSyncClass extends AbstractSyncTool { + public ValidMetaSyncClass(TypedProperties props, Configuration conf, FileSystem fs) { + super(props, conf, fs); + } + + @Override + public void syncHoodieTable() { + throw new HoodieException("Method unimplemented as its a test class"); + } + } + + public static class DeprecatedMetaSyncClass extends AbstractSyncTool { + public DeprecatedMetaSyncClass(Properties props, FileSystem fileSystem) { + super(props, fileSystem); + } + + @Override + public void syncHoodieTable() { + throw new HoodieException("Method unimplemented as its a test class"); + } + } + + public static class InvalidSyncClass { + public InvalidSyncClass(Properties props) { + } + } +} diff --git a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties new file mode 100644 index 0000000000000..b21b5d4070c41 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties @@ -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. +### +log4j.rootLogger=WARN, CONSOLE +log4j.logger.org.apache.hudi=DEBUG + +# 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 diff --git a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties new file mode 100644 index 0000000000000..c03e808cca1f8 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.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=INFO +log4j.logger.org.apache.hudi=DEBUG + +# 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 \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java index 833fce295e326..84b7933767610 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -18,7 +18,6 @@ package org.apache.hudi.utilities.deltastreamer; -import org.apache.hudi.DataSourceUtils; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -34,13 +33,13 @@ import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -161,12 +160,16 @@ public void execute() throws IOException { */ private void syncHive() { if (cfg.enableHiveSync || cfg.enableMetaSync) { - HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat); - HiveConf hiveConf = new HiveConf(fs.getConf(), HiveConf.class); - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname,hiveSyncConfig.metastoreUris); - LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString()); - LOG.info("Hive Sync Conf => " + hiveSyncConfig); - new HiveSyncTool(hiveSyncConfig, new HiveConf(configuration, HiveConf.class), fs).syncHoodieTable(); + TypedProperties metaProps = new TypedProperties(); + metaProps.putAll(props); + metaProps.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), cfg.targetBasePath); + metaProps.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat); + if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) { + metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), + props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key()))); + } + + new HiveSyncTool(metaProps, configuration, fs).syncHoodieTable(); } } @@ -195,4 +198,4 @@ private void initializeTable() throws IOException { public HoodieWriteConfig getBootstrapConfig() { return bootstrapConfig; } -} \ No newline at end of file +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 7dd9a36892cf2..50338e5510c5b 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 @@ -42,12 +42,12 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; 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.StringUtils; 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.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -59,7 +59,7 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metrics.HoodieMetrics; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.util.SyncUtilHelpers; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig; @@ -81,7 +81,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -99,7 +98,6 @@ import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Properties; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -692,44 +690,24 @@ private void syncMeta(HoodieDeltaStreamerMetrics metrics) { LOG.info("When set --enable-hive-sync will use HiveSyncTool for backward compatibility"); } if (cfg.enableMetaSync) { + FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); + + TypedProperties metaProps = new TypedProperties(); + metaProps.putAll(props); + if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) { + metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), + props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key()))); + } + for (String impl : syncClientToolClasses) { Timer.Context syncContext = metrics.getMetaSyncTimerContext(); - impl = impl.trim(); - switch (impl) { - case "org.apache.hudi.hive.HiveSyncTool": - syncHive(); - break; - default: - FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()); - Properties properties = new Properties(); - properties.putAll(props); - properties.put("basePath", cfg.targetBasePath); - properties.put("baseFileFormat", cfg.baseFileFormat); - AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[]{Properties.class, FileSystem.class}, properties, fs); - syncTool.syncHoodieTable(); - } + SyncUtilHelpers.runHoodieMetaSync(impl.trim(), metaProps, conf, fs, cfg.targetBasePath, cfg.baseFileFormat); long metaSyncTimeMs = syncContext != null ? syncContext.stop() : 0; metrics.updateDeltaStreamerMetaSyncMetrics(getSyncClassShortName(impl), metaSyncTimeMs); } } } - public void syncHive() { - HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat); - HiveConf hiveConf = new HiveConf(conf, HiveConf.class); - if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) { - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris); - } - LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString()); - LOG.info("Hive Sync Conf => " + hiveSyncConfig.toString()); - new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable(); - } - - public void syncHive(HiveConf conf) { - this.conf = conf; - syncHive(); - } - /** * Note that depending on configs and source-type, schemaProvider could either be eagerly or lazily created. * SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index bcd7b3b7d8ac6..3c72eb58af1e8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -19,7 +19,6 @@ package org.apache.hudi.utilities.deltastreamer; import com.beust.jcommander.Parameter; -import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.client.utils.OperationConverter; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; @@ -28,6 +27,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.utilities.IdentitySplitter; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.utilities.UtilHelpers; @@ -131,7 +131,7 @@ private void populateTableExecutionContextList(TypedProperties properties, Strin Helpers.deepCopyConfigs(config, cfg); String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, ""); cfg.targetBasePath = StringUtils.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath; - if (cfg.enableMetaSync && StringUtils.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE().key(), ""))) { + if (cfg.enableMetaSync && StringUtils.isNullOrEmpty(tableProperties.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), ""))) { throw new HoodieException("Meta sync table field not provided!"); } populateSchemaProviderProps(cfg, tableProperties); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java index a2f70e0416942..f6ea5c0f55485 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java @@ -20,10 +20,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.ql.metadata.HiveException; + +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.HoodieHiveClient; import org.apache.hudi.hive.testutils.HiveTestUtil; +import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.utilities.exception.HoodieIncrementalPullSQLException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -38,7 +41,7 @@ import java.time.Instant; import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; -import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncConfig; +import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -69,12 +72,12 @@ public void testInitHiveIncrementalPuller() { } private HiveIncrementalPuller.Config getHivePullerConfig(String incrementalSql) throws IOException { - config.hiveJDBCUrl = hiveSyncConfig.jdbcUrl; - config.hiveUsername = hiveSyncConfig.hiveUser; - config.hivePassword = hiveSyncConfig.hivePass; + config.hiveJDBCUrl = hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key()); + config.hiveUsername = hiveSyncProps.getString(HiveSyncConfig.HIVE_USER.key()); + config.hivePassword = hiveSyncProps.getString(HiveSyncConfig.HIVE_PASS.key()); config.hoodieTmpDir = Files.createTempDirectory("hivePullerTest").toUri().toString(); - config.sourceDb = hiveSyncConfig.databaseName; - config.sourceTable = hiveSyncConfig.tableName; + config.sourceDb = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()); + config.sourceTable = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key()); config.targetDb = "tgtdb"; config.targetTable = "test2"; config.tmpDb = "tmp_db"; @@ -98,9 +101,9 @@ private void createIncrementalSqlFile(String text, HiveIncrementalPuller.Config private void createSourceTable() throws IOException, URISyntaxException { String instantTime = "101"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncConfig.syncMode = "jdbc"; - HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; - HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc"); + + HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); tool.syncHoodieTable(); } @@ -113,20 +116,20 @@ private void createTargetTable() throws IOException, URISyntaxException { tool.syncHoodieTable(); } - private HiveSyncConfig getTargetHiveSyncConfig(String basePath) { - HiveSyncConfig config = HiveSyncConfig.copy(hiveSyncConfig); - config.databaseName = "tgtdb"; - config.tableName = "test2"; - config.basePath = basePath; - config.batchSyncNum = 3; - config.syncMode = "jdbc"; - return config; + private TypedProperties getTargetHiveSyncConfig(String basePath) { + TypedProperties targetHiveSyncProps = new TypedProperties(hiveSyncProps); + targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "tgtdb"); + targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "test2"); + targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), basePath); + targetHiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc"); + + return targetHiveSyncProps; } - private HiveSyncConfig getAssertionSyncConfig(String databaseName) { - HiveSyncConfig config = HiveSyncConfig.copy(hiveSyncConfig); - config.databaseName = databaseName; - return config; + private TypedProperties getAssertionSyncConfig(String databaseName) { + TypedProperties assertHiveSyncProps = new TypedProperties(hiveSyncProps); + assertHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), databaseName); + return assertHiveSyncProps; } private void createTables() throws IOException, URISyntaxException { @@ -158,12 +161,11 @@ public void testPullerWithoutSourceInSql() throws IOException, URISyntaxExceptio public void testPuller() throws IOException, URISyntaxException { createTables(); HiveIncrementalPuller.Config cfg = getHivePullerConfig("select name from testdb.test1 where `_hoodie_commit_time` > '%s'"); - HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + HoodieHiveClient hiveClient = new HoodieHiveClient(new HiveSyncConfig(hiveSyncProps), HiveTestUtil.getHiveConf(), fileSystem); hiveClient.createDatabase(cfg.tmpDb); HiveIncrementalPuller puller = new HiveIncrementalPuller(cfg); puller.saveDelta(); - HiveSyncConfig assertingConfig = getAssertionSyncConfig(cfg.tmpDb); - HoodieHiveClient assertingClient = new HoodieHiveClient(assertingConfig, HiveTestUtil.getHiveConf(), fileSystem); + HoodieHiveClient assertingClient = new HoodieHiveClient(new HiveSyncConfig(getAssertionSyncConfig(cfg.tmpDb)), HiveTestUtil.getHiveConf(), fileSystem); String tmpTable = cfg.targetTable + "__" + cfg.sourceTable; assertTrue(assertingClient.doesTableExist(tmpTable)); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java index 02b1848e2e31e..a4d91f2a50ade 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java @@ -18,7 +18,6 @@ package org.apache.hudi.utilities.functional; -import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; @@ -28,6 +27,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.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; @@ -178,11 +178,11 @@ protected static void writeCommonPropsToFile(FileSystem dfs, String dfsBasePath) props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc"); // Hive Configs - props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1"); - props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "hive_trips"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1"); + props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "hive_trips"); + props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); + props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getName()); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE); } @@ -237,11 +237,11 @@ protected static void populateCommonKafkaProps(TypedProperties props, String bro protected static void populateCommonHiveProps(TypedProperties props) { // Hive Configs - props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb2"); - props.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION().key(), "false"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr"); - props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), + props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb2"); + props.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "false"); + props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); + props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getName()); }