diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 29702846b3d2d..b76a465d7128c 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -59,3 +59,11 @@ jobs: if: ${{ !endsWith(env.SPARK_PROFILE, '3.2') }} # skip test spark 3.2 before hadoop upgrade to 3.x run: mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -D"$FLINK_PROFILE" -DfailIfNoTests=false -pl hudi-examples/hudi-examples-flink,hudi-examples/hudi-examples-java,hudi-examples/hudi-examples-spark + - name: Spark SQL Test + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + FLINK_PROFILE: ${{ matrix.flinkProfile }} + if: ${{ !endsWith(env.SPARK_PROFILE, '2.4') }} # skip test spark 2.4 as it's covered by Azure CI + run: + mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -D"$FLINK_PROFILE" '-Dtest=org.apache.spark.sql.hudi.Test*' -pl hudi-spark-datasource/hudi-spark diff --git a/docker/demo/config/test-suite/deltastreamer-immutable-dataset.yaml b/docker/demo/config/test-suite/deltastreamer-immutable-dataset.yaml new file mode 100644 index 0000000000000..4903e3650c144 --- /dev/null +++ b/docker/demo/config/test-suite/deltastreamer-immutable-dataset.yaml @@ -0,0 +1,53 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-immutable-dataset.yaml +dag_rounds: 5 +dag_intermittent_delay_mins: 0 +dag_content: + first_bulk_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 3 + num_records_insert: 5000 + type: BulkInsertNode + deps: none + first_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_bulk_insert + first_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 3 + num_records_insert: 5000 + type: InsertNode + deps: first_validate + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_insert + last_validate: + config: + execute_itr_count: 5 + delete_input_data: true + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/docker/demo/config/test-suite/deltastreamer-pure-bulk-inserts.yaml b/docker/demo/config/test-suite/deltastreamer-pure-bulk-inserts.yaml new file mode 100644 index 0000000000000..d5342e22b1282 --- /dev/null +++ b/docker/demo/config/test-suite/deltastreamer-pure-bulk-inserts.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-pure-bulk-inserts.yaml +dag_rounds: 10 +dag_intermittent_delay_mins: 0 +dag_content: + first_bulk_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 3 + num_records_insert: 5000 + type: BulkInsertNode + deps: none + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_bulk_insert + last_validate: + config: + execute_itr_count: 10 + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/docker/demo/config/test-suite/deltastreamer-pure-inserts.yaml b/docker/demo/config/test-suite/deltastreamer-pure-inserts.yaml new file mode 100644 index 0000000000000..3b209fe5fe016 --- /dev/null +++ b/docker/demo/config/test-suite/deltastreamer-pure-inserts.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-pure-inserts.yaml +dag_rounds: 10 +dag_intermittent_delay_mins: 0 +dag_content: + first_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 3 + num_records_insert: 5000 + type: InsertNode + deps: none + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_insert + last_validate: + config: + execute_itr_count: 10 + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/docker/demo/config/test-suite/insert-overwrite.yaml b/docker/demo/config/test-suite/insert-overwrite.yaml index dc185d5938f6d..7e54cea6a910d 100644 --- a/docker/demo/config/test-suite/insert-overwrite.yaml +++ b/docker/demo/config/test-suite/insert-overwrite.yaml @@ -17,7 +17,6 @@ dag_name: simple-deltastreamer.yaml dag_rounds: 1 dag_intermittent_delay_mins: 1 dag_content: - first_insert: config: record_size: 1000 @@ -91,4 +90,4 @@ dag_content: validate_hive: false delete_input_data: false type: ValidateDatasetNode - deps: third_upsert + deps: third_upsert \ No newline at end of file diff --git a/docker/demo/config/test-suite/multi-writer-1-ds.yaml b/docker/demo/config/test-suite/multi-writer-1-ds.yaml index 3fe33b671dc39..3476d8075a6ed 100644 --- a/docker/demo/config/test-suite/multi-writer-1-ds.yaml +++ b/docker/demo/config/test-suite/multi-writer-1-ds.yaml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. dag_name: simple-deltastreamer.yaml -dag_rounds: 3 +dag_rounds: 6 dag_intermittent_delay_mins: 0 dag_content: first_insert: diff --git a/docker/demo/config/test-suite/multi-writer-1-sds.yaml b/docker/demo/config/test-suite/multi-writer-1-sds.yaml new file mode 100644 index 0000000000000..d60a8ba6d78a6 --- /dev/null +++ b/docker/demo/config/test-suite/multi-writer-1-sds.yaml @@ -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. +dag_name: cow-spark-simple.yaml +dag_rounds: 6 +dag_intermittent_delay_mins: 0 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100000 + start_partition: 1 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 1 + num_records_insert: 50000 + repeat_count: 1 + num_records_upsert: 50000 + num_partitions_upsert: 1 + start_partition: 1 + type: SparkUpsertNode + deps: first_insert + first_delete: + config: + num_partitions_delete: 0 + num_records_delete: 10000 + start_partition: 1 + type: SparkDeleteNode + deps: first_upsert + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/multi-writer-2-sds.yaml b/docker/demo/config/test-suite/multi-writer-2-sds.yaml index 9242dd26051ec..702065c672112 100644 --- a/docker/demo/config/test-suite/multi-writer-2-sds.yaml +++ b/docker/demo/config/test-suite/multi-writer-2-sds.yaml @@ -14,8 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. dag_name: cow-spark-simple.yaml -dag_rounds: 3 -dag_intermittent_delay_mins: 0 +dag_rounds: 5 +dag_intermittent_delay_mins: 1 dag_content: first_insert: config: diff --git a/docker/demo/config/test-suite/multi-writer-3-sds.yaml b/docker/demo/config/test-suite/multi-writer-3-sds.yaml new file mode 100644 index 0000000000000..9ad21f467d50b --- /dev/null +++ b/docker/demo/config/test-suite/multi-writer-3-sds.yaml @@ -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. +dag_name: cow-spark-simple.yaml +dag_rounds: 4 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100000 + start_partition: 20 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 1 + num_records_insert: 50000 + repeat_count: 1 + num_records_upsert: 50000 + num_partitions_upsert: 1 + start_partition: 20 + type: SparkUpsertNode + deps: first_insert + first_delete: + config: + num_partitions_delete: 0 + num_records_delete: 10000 + start_partition: 20 + type: SparkDeleteNode + deps: first_upsert + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/multi-writer-4-sds.yaml b/docker/demo/config/test-suite/multi-writer-4-sds.yaml new file mode 100644 index 0000000000000..74dfa1cb4ba6a --- /dev/null +++ b/docker/demo/config/test-suite/multi-writer-4-sds.yaml @@ -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. +dag_name: cow-spark-simple.yaml +dag_rounds: 4 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100000 + start_partition: 30 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 1 + num_records_insert: 50000 + repeat_count: 1 + num_records_upsert: 50000 + num_partitions_upsert: 1 + start_partition: 30 + type: SparkUpsertNode + deps: first_insert + first_delete: + config: + num_partitions_delete: 0 + num_records_delete: 10000 + start_partition: 30 + type: SparkDeleteNode + deps: first_upsert + second_validate: + config: + validate_hive: false + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/multi-writer-local-3.properties b/docker/demo/config/test-suite/multi-writer-local-3.properties new file mode 100644 index 0000000000000..48f0f0b1ace8b --- /dev/null +++ b/docker/demo/config/test-suite/multi-writer-local-3.properties @@ -0,0 +1,57 @@ + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +hoodie.insert.shuffle.parallelism=2 +hoodie.upsert.shuffle.parallelism=2 +hoodie.bulkinsert.shuffle.parallelism=2 +hoodie.delete.shuffle.parallelism=2 + +hoodie.metadata.enable=false + +hoodie.deltastreamer.source.test.num_partitions=100 +hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false +hoodie.deltastreamer.source.test.max_unique_records=100000000 +hoodie.embed.timeline.server=false +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector + +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.datasource.hive_sync.skip_ro_suffix=true + +hoodie.datasource.write.recordkey.field=_row_key +hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator +hoodie.datasource.write.partitionpath.field=timestamp + +hoodie.write.concurrency.mode=optimistic_concurrency_control +hoodie.cleaner.policy.failed.writes=LAZY +hoodie.write.lock.provider=org.apache.hudi.client.transaction.lock.InProcessLockProvider + +hoodie.deltastreamer.source.dfs.root=/tmp/hudi/input3 +hoodie.deltastreamer.schemaprovider.target.schema.file=file:/tmp/source.avsc +hoodie.deltastreamer.schemaprovider.source.schema.file=file:/tmp/source.avsc +hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP +hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd + +hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/ +hoodie.datasource.hive_sync.database=testdb +hoodie.datasource.hive_sync.table=table1 +hoodie.datasource.hive_sync.assume_date_partitioning=false +hoodie.datasource.hive_sync.partition_fields=_hoodie_partition_path +hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor + diff --git a/docker/demo/config/test-suite/multi-writer-local-4.properties b/docker/demo/config/test-suite/multi-writer-local-4.properties new file mode 100644 index 0000000000000..4b5120928ccb1 --- /dev/null +++ b/docker/demo/config/test-suite/multi-writer-local-4.properties @@ -0,0 +1,57 @@ + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +hoodie.insert.shuffle.parallelism=2 +hoodie.upsert.shuffle.parallelism=2 +hoodie.bulkinsert.shuffle.parallelism=2 +hoodie.delete.shuffle.parallelism=2 + +hoodie.metadata.enable=false + +hoodie.deltastreamer.source.test.num_partitions=100 +hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false +hoodie.deltastreamer.source.test.max_unique_records=100000000 +hoodie.embed.timeline.server=false +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector + +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.datasource.hive_sync.skip_ro_suffix=true + +hoodie.datasource.write.recordkey.field=_row_key +hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator +hoodie.datasource.write.partitionpath.field=timestamp + +hoodie.write.concurrency.mode=optimistic_concurrency_control +hoodie.cleaner.policy.failed.writes=LAZY +hoodie.write.lock.provider=org.apache.hudi.client.transaction.lock.InProcessLockProvider + +hoodie.deltastreamer.source.dfs.root=/tmp/hudi/input4 +hoodie.deltastreamer.schemaprovider.target.schema.file=file:/tmp/source.avsc +hoodie.deltastreamer.schemaprovider.source.schema.file=file:/tmp/source.avsc +hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP +hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd + +hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/ +hoodie.datasource.hive_sync.database=testdb +hoodie.datasource.hive_sync.table=table1 +hoodie.datasource.hive_sync.assume_date_partitioning=false +hoodie.datasource.hive_sync.partition_fields=_hoodie_partition_path +hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor + diff --git a/docker/demo/config/test-suite/spark-delete-partition.yaml b/docker/demo/config/test-suite/spark-delete-partition.yaml new file mode 100644 index 0000000000000..1d23fa7b0851c --- /dev/null +++ b/docker/demo/config/test-suite/spark-delete-partition.yaml @@ -0,0 +1,57 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-delete-partition.yaml +dag_rounds: 1 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 10 + type: SparkInsertNode + deps: none + first_delete_partition: + config: + partitions_to_delete: "1970/01/01" + type: SparkDeletePartitionNode + deps: first_insert + second_validate: + config: + validate_full_data : true + input_partitions_to_skip_validate : "0" + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_delete_partition + second_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 10 + start_partition: 2 + type: SparkInsertNode + deps: second_validate + third_validate: + config: + validate_full_data : true + input_partitions_to_skip_validate : "0" + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: second_insert \ No newline at end of file diff --git a/docker/demo/config/test-suite/spark-immutable-dataset.yaml b/docker/demo/config/test-suite/spark-immutable-dataset.yaml new file mode 100644 index 0000000000000..d6cbf1b244de5 --- /dev/null +++ b/docker/demo/config/test-suite/spark-immutable-dataset.yaml @@ -0,0 +1,53 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-immutable-dataset.yaml +dag_rounds: 5 +dag_intermittent_delay_mins: 0 +dag_content: + first_bulk_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 5 + num_records_insert: 5000 + type: SparkBulkInsertNode + deps: none + first_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_bulk_insert + first_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 5 + num_records_insert: 5000 + type: SparkInsertNode + deps: first_validate + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_insert + last_validate: + config: + execute_itr_count: 5 + delete_input_data: true + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml index 3c47729e66470..dfbfba0a15700 100644 --- a/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml +++ b/docker/demo/config/test-suite/spark-long-running-non-partitioned.yaml @@ -14,24 +14,24 @@ # See the License for the specific language governing permissions and # limitations under the License. dag_name: cow-spark-deltastreamer-long-running-multi-partitions.yaml -dag_rounds: 6 -dag_intermittent_delay_mins: 0 +dag_rounds: 10 +dag_intermittent_delay_mins: 1 dag_content: first_insert: config: record_size: 200 num_partitions_insert: 1 repeat_count: 1 - num_records_insert: 10000 + num_records_insert: 1000 type: SparkInsertNode deps: none first_upsert: config: record_size: 200 num_partitions_insert: 1 - num_records_insert: 300 + num_records_insert: 1000 repeat_count: 1 - num_records_upsert: 3000 + num_records_upsert: 1000 num_partitions_upsert: 1 type: SparkUpsertNode deps: first_insert @@ -43,7 +43,6 @@ dag_content: deps: first_upsert second_validate: config: - validate_once_every_itr : 3 validate_hive: false delete_input_data: true type: ValidateDatasetNode diff --git a/docker/demo/config/test-suite/spark-non-core-operations.yaml b/docker/demo/config/test-suite/spark-non-core-operations.yaml new file mode 100644 index 0000000000000..f7189ce4587c8 --- /dev/null +++ b/docker/demo/config/test-suite/spark-non-core-operations.yaml @@ -0,0 +1,204 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-non-core-operations.yaml +dag_rounds: 1 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: none + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: first_insert + second_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: first_upsert + second_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: second_insert + first_insert_overwrite: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10 + type: SparkInsertOverwriteNode + deps: second_upsert + delete_all_input_except_last: + config: + delete_input_data_except_latest: true + type: DeleteInputDatasetNode + deps: first_insert_overwrite + third_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: delete_all_input_except_last + third_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: third_insert + second_validate: + config: + validate_full_data : true + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: third_upsert + fourth_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: second_validate + fourth_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: fourth_insert + fifth_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: fourth_upsert + fifth_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: fifth_insert + first_insert_overwrite_table: + config: + record_size: 1000 + repeat_count: 1 + num_records_insert: 10 + type: SparkInsertOverwriteTableNode + deps: fifth_upsert + second_delete_all_input_except_last: + config: + delete_input_data_except_latest: true + type: DeleteInputDatasetNode + deps: first_insert_overwrite_table + sixth_insert: + config: + record_size: 1000 + num_partitions_insert: 10 + repeat_count: 1 + num_records_insert: 10000 + type: SparkInsertNode + deps: second_delete_all_input_except_last + sixth_upsert: + config: + record_size: 1000 + num_partitions_insert: 10 + num_records_insert: 1000 + repeat_count: 1 + num_records_upsert: 8000 + num_partitions_upsert: 10 + type: SparkUpsertNode + deps: sixth_insert + third_validate: + config: + validate_full_data : true + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: sixth_upsert + seventh_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 10 + type: SparkInsertNode + deps: third_validate + first_delete_partition: + config: + partitions_to_delete: "1970/01/01" + type: SparkDeletePartitionNode + deps: seventh_insert + fourth_validate: + config: + validate_full_data : true + input_partitions_to_skip_validate : "0" + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_delete_partition + eigth_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 10 + start_partition: 2 + type: SparkInsertNode + deps: fourth_validate + fifth_validate: + config: + validate_full_data : true + input_partitions_to_skip_validate : "0" + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: eigth_insert \ No newline at end of file diff --git a/docker/demo/config/test-suite/spark-pure-bulk-inserts.yaml b/docker/demo/config/test-suite/spark-pure-bulk-inserts.yaml new file mode 100644 index 0000000000000..f82705cea3cec --- /dev/null +++ b/docker/demo/config/test-suite/spark-pure-bulk-inserts.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-pure-bulk-inserts.yaml +dag_rounds: 5 +dag_intermittent_delay_mins: 0 +dag_content: + first_bulk_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 4 + num_records_insert: 5000 + type: SparkBulkInsertNode + deps: none + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_bulk_insert + last_validate: + config: + execute_itr_count: 5 + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/docker/demo/config/test-suite/spark-pure-inserts.yaml b/docker/demo/config/test-suite/spark-pure-inserts.yaml new file mode 100644 index 0000000000000..13482f988c70c --- /dev/null +++ b/docker/demo/config/test-suite/spark-pure-inserts.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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-pure-inserts.yaml +dag_rounds: 5 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 200 + num_partitions_insert: 10 + repeat_count: 3 + num_records_insert: 5000 + type: SparkInsertNode + deps: none + second_validate: + config: + validate_hive: false + delete_input_data: false + type: ValidateDatasetNode + deps: first_insert + last_validate: + config: + execute_itr_count: 10 + type: ValidateAsyncOperations + deps: second_validate \ No newline at end of file diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java index 81c05ed132a35..e5a23a9a571cc 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java @@ -394,7 +394,7 @@ public void createDatabase(String databaseName) { public Option getLastCommitTimeSynced(String tableName) { try { Table table = getTable(awsGlue, databaseName, tableName); - return Option.of(table.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); + return Option.ofNullable(table.getParameters().get(HOODIE_LAST_COMMIT_TIME_SYNC)); } catch (Exception e) { throw new HoodieGlueSyncException("Fail to get last sync commit time for " + tableId(databaseName, tableName), e); } diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala index 3802bb46a0f5b..fbfc1d8ec902e 100644 --- a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala +++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala @@ -28,7 +28,7 @@ import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory} import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.util.BaseFileUtils import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig} -import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter} +import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieAvroParquetWriter} import org.apache.parquet.avro.AvroSchemaConverter import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.spark.sql.{DataFrame, SQLContext} @@ -50,8 +50,7 @@ object SparkHelpers { // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'. parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader) - val writer = new HoodieParquetWriter[HoodieJsonPayload, IndexedRecord](instantTime, destinationFile, parquetConfig, schema, new SparkTaskContextSupplier(), - true) + val writer = new HoodieAvroParquetWriter[IndexedRecord](destinationFile, parquetConfig, instantTime, new SparkTaskContextSupplier(), true) for (rec <- sourceRecords) { val key: String = rec.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString if (!keysToSkip.contains(key)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java index 3f208a0f86a09..b41747d83a85e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java @@ -135,7 +135,8 @@ protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoa return HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) + .setProperties(config.getProps()).build(); } public Option getTimelineServer() { 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 4b747d3a77c00..270027df18053 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 @@ -334,7 +334,7 @@ protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata met * @param metadata instance of {@link HoodieCommitMetadata}. */ protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) { - context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table"); + context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table: " + config.getTableName()); table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime, table.isTableServiceAction(actionType))); } @@ -1038,7 +1038,7 @@ public void dropIndex(List partitionTypes) { HoodieInstant ownerInstant = new HoodieInstant(true, HoodieTimeline.INDEXING_ACTION, dropInstant); this.txnManager.beginTransaction(Option.of(ownerInstant), Option.empty()); try { - context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table"); + context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table: " + config.getTableName()); table.getMetadataWriter(dropInstant).ifPresent(w -> { try { ((HoodieTableMetadataWriter) w).dropMetadataPartitions(partitionTypes); @@ -1466,8 +1466,6 @@ protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { + protected void tryUpgrade(HoodieTableMetaClient metaClient, Option instantTime) { UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java index 40e8f85a3ac70..d006b52b3306a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java @@ -85,7 +85,7 @@ public List validateCompactionPlan(HoodieTableMetaClient met if (plan.getOperations() != null) { List ops = plan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); - context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations"); + context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations: " + config.getTableName()); return context.map(ops, op -> { try { return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView)); @@ -351,7 +351,7 @@ private List runRenamingOps(HoodieTableMetaClient metaClient, } else { LOG.info("The following compaction renaming operations needs to be performed to un-schedule"); if (!dryRun) { - context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations"); + context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations: " + config.getTableName()); return context.map(renameActions, lfPair -> { try { LOG.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath()); @@ -394,7 +394,7 @@ public List> getRenamingActionsForUnschedulin "Number of Compaction Operations :" + plan.getOperations().size() + " for instant :" + compactionInstant); List ops = plan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); - context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations"); + context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations: " + config.getTableName()); return context.flatMap(ops, op -> { try { return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, 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 190a5fe1c6064..2974cc2ef6d6f 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 @@ -459,6 +459,9 @@ private Stream getCommitInstantsToArchive() { private Stream getInstantsToArchive() { Stream instants = Stream.concat(getCleanInstantsToArchive(), getCommitInstantsToArchive()); + if (config.isMetastoreEnabled()) { + return Stream.empty(); + } // For archiving and cleaning instants, we need to include intermediate state files if they exist HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); @@ -519,7 +522,7 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo new Path(metaClient.getMetaPath(), archivedInstant.getFileName()) ).map(Path::toString).collect(Collectors.toList()); - context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants"); + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); Map resultDeleteInstantFiles = deleteFilesParallelize(metaClient, instantFiles, context, false); for (Map.Entry result : resultDeleteInstantFiles.entrySet()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java index 020944e7ab9b1..ad54f8c0a0992 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java @@ -45,7 +45,7 @@ public LazyIterableIterator(Iterator in) { /** * Called once, before any elements are processed. */ - protected abstract void start(); + protected void start() {} /** * Block computation to be overwritten by sub classes. @@ -55,7 +55,7 @@ public LazyIterableIterator(Iterator in) { /** * Called once, after all elements are processed. */ - protected abstract void end(); + protected void end() {} ////////////////// // iterable implementation diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java index 3d7e3a7941daa..2389aa7fc1e02 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java @@ -157,6 +157,33 @@ public class HoodieHBaseIndexConfig extends HoodieConfig { .withDocumentation("When set to true, the rollback method will delete the last failed task index. " + "The default value is false. Because deleting the index will add extra load on the Hbase cluster for each rollback"); + public static final ConfigProperty SECURITY_AUTHENTICATION = ConfigProperty + .key("hoodie.index.hbase.security.authentication") + .defaultValue("simple") + .withDocumentation("Property to decide if the hbase cluster secure authentication is enabled or not. " + + "Possible values are 'simple' (no authentication), and 'kerberos'."); + + public static final ConfigProperty KERBEROS_USER_KEYTAB = ConfigProperty + .key("hoodie.index.hbase.kerberos.user.keytab") + .noDefaultValue() + .withDocumentation("File name of the kerberos keytab file for connecting to the hbase cluster."); + + public static final ConfigProperty KERBEROS_USER_PRINCIPAL = ConfigProperty + .key("hoodie.index.hbase.kerberos.user.principal") + .noDefaultValue() + .withDocumentation("The kerberos principal name for connecting to the hbase cluster."); + + public static final ConfigProperty REGIONSERVER_PRINCIPAL = ConfigProperty + .key("hoodie.index.hbase.regionserver.kerberos.principal") + .noDefaultValue() + .withDocumentation("The value of hbase.regionserver.kerberos.principal in hbase cluster."); + + public static final ConfigProperty MASTER_PRINCIPAL = ConfigProperty + .key("hoodie.index.hbase.master.kerberos.principal") + .noDefaultValue() + .withDocumentation("The value of hbase.master.kerberos.principal in hbase cluster."); + + /** * @deprecated Use {@link #ZKQUORUM} and its methods instead */ @@ -444,6 +471,31 @@ public Builder hbaseZkZnodeParent(String zkZnodeParent) { return this; } + public Builder hbaseSecurityAuthentication(String authentication) { + hBaseIndexConfig.setValue(SECURITY_AUTHENTICATION, authentication); + return this; + } + + public Builder hbaseKerberosUserKeytab(String keytab) { + hBaseIndexConfig.setValue(KERBEROS_USER_KEYTAB, keytab); + return this; + } + + public Builder hbaseKerberosUserPrincipal(String principal) { + hBaseIndexConfig.setValue(KERBEROS_USER_PRINCIPAL, principal); + return this; + } + + public Builder hbaseKerberosRegionserverPrincipal(String principal) { + hBaseIndexConfig.setValue(REGIONSERVER_PRINCIPAL, principal); + return this; + } + + public Builder hbaseKerberosMasterPrincipal(String principal) { + hBaseIndexConfig.setValue(MASTER_PRINCIPAL, principal); + return this; + } + /** *

* Method to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 7c1f7e00e7fb1..dbd45b9738285 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -216,19 +216,40 @@ public class HoodieIndexConfig extends HoodieConfig { /** * ***** Bucket Index Configs ***** * Bucket Index is targeted to locate the record fast by hash in big data scenarios. - * The current implementation is a basic version, so there are some constraints: - * 1. Unsupported operation: bulk insert, cluster and so on. - * 2. Bucket num change requires rewriting the partition. - * 3. Predict the table size and future data growth well to set a reasonable bucket num. - * 4. A bucket size is recommended less than 3GB and avoid bing too small. - * more details and progress see [HUDI-3039]. - */ - // Bucket num equals file groups num in each partition. - // Bucket num can be set according to partition size and file group size. + * A bucket size is recommended less than 3GB to avoid being too small. + * For more details and progress, see [HUDI-3039]. + */ + + /** + * Bucket Index Engine Type: implementation of bucket index + * + * SIMPLE: + * 0. Check `HoodieSimpleBucketLayout` for its supported operations. + * 1. Bucket num is fixed and requires rewriting the partition if we want to change it. + * + * CONSISTENT_HASHING: + * 0. Check `HoodieConsistentBucketLayout` for its supported operations. + * 1. Bucket num will auto-adjust by running clustering (still in progress) + */ + public static final ConfigProperty BUCKET_INDEX_ENGINE_TYPE = ConfigProperty + .key("hoodie.index.bucket.engine") + .defaultValue("SIMPLE") + .sinceVersion("0.11.0") + .withDocumentation("Type of bucket index engine to use. Default is SIMPLE bucket index, with fixed number of bucket." + + "Possible options are [SIMPLE | CONSISTENT_HASHING]." + + "Consistent hashing supports dynamic resizing of the number of bucket, solving potential data skew and file size " + + "issues of the SIMPLE hashing engine."); + + /** + * Bucket num equals file groups num in each partition. + * Bucket num can be set according to partition size and file group size. + * + * In dynamic bucket index cases (e.g., using CONSISTENT_HASHING), this config of number of bucket serves as a initial bucket size + */ public static final ConfigProperty BUCKET_INDEX_NUM_BUCKETS = ConfigProperty .key("hoodie.bucket.index.num.buckets") .defaultValue(256) - .withDocumentation("Only applies if index type is BUCKET_INDEX. Determine the number of buckets in the hudi table, " + .withDocumentation("Only applies if index type is BUCKET. Determine the number of buckets in the hudi table, " + "and each partition is divided to N buckets."); public static final ConfigProperty BUCKET_INDEX_HASH_FIELD = ConfigProperty @@ -463,6 +484,11 @@ public Builder withIndexType(HoodieIndex.IndexType indexType) { return this; } + public Builder withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType bucketType) { + hoodieIndexConfig.setValue(BUCKET_INDEX_ENGINE_TYPE, bucketType.name()); + return this; + } + public Builder withIndexClass(String indexClass) { hoodieIndexConfig.setValue(INDEX_CLASS_NAME, indexClass); 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 7b49a7a466785..dd5c0bfd6ded3 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 @@ -28,6 +28,7 @@ import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.HoodieMetastoreConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; @@ -495,6 +496,7 @@ public class HoodieWriteConfig extends HoodieConfig { private FileSystemViewStorageConfig viewStorageConfig; private HoodiePayloadConfig hoodiePayloadConfig; private HoodieMetadataConfig metadataConfig; + private HoodieMetastoreConfig metastoreConfig; private HoodieCommonConfig commonConfig; private EngineType engineType; @@ -886,6 +888,7 @@ protected HoodieWriteConfig(EngineType engineType, Properties props) { this.viewStorageConfig = clientSpecifiedViewStorageConfig; this.hoodiePayloadConfig = HoodiePayloadConfig.newBuilder().fromProperties(newProps).build(); this.metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(props).build(); + this.metastoreConfig = HoodieMetastoreConfig.newBuilder().fromProperties(props).build(); this.commonConfig = HoodieCommonConfig.newBuilder().fromProperties(props).build(); } @@ -1428,6 +1431,10 @@ public String getIndexClass() { return getString(HoodieIndexConfig.INDEX_CLASS_NAME); } + public HoodieIndex.BucketIndexEngineType getBucketIndexEngineType() { + return HoodieIndex.BucketIndexEngineType.valueOf(getString(HoodieIndexConfig.BUCKET_INDEX_ENGINE_TYPE)); + } + public int getBloomFilterNumEntries() { return getInt(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE); } @@ -1488,6 +1495,26 @@ public boolean getHBaseIndexShouldComputeQPSDynamically() { return getBoolean(HoodieHBaseIndexConfig.COMPUTE_QPS_DYNAMICALLY); } + public String getHBaseIndexSecurityAuthentication() { + return getString(HoodieHBaseIndexConfig.SECURITY_AUTHENTICATION); + } + + public String getHBaseIndexKerberosUserKeytab() { + return getString(HoodieHBaseIndexConfig.KERBEROS_USER_KEYTAB); + } + + public String getHBaseIndexKerberosUserPrincipal() { + return getString(HoodieHBaseIndexConfig.KERBEROS_USER_PRINCIPAL); + } + + public String getHBaseIndexRegionserverPrincipal() { + return getString(HoodieHBaseIndexConfig.REGIONSERVER_PRINCIPAL); + } + + public String getHBaseIndexMasterPrincipal() { + return getString(HoodieHBaseIndexConfig.MASTER_PRINCIPAL); + } + public int getHBaseIndexDesiredPutsTime() { return getInt(HoodieHBaseIndexConfig.DESIRED_PUTS_TIME_IN_SECONDS); } @@ -2076,6 +2103,13 @@ public HoodieStorageLayout.LayoutType getLayoutType() { return HoodieStorageLayout.LayoutType.valueOf(getString(HoodieLayoutConfig.LAYOUT_TYPE)); } + /** + * Metastore configs. + */ + public boolean isMetastoreEnabled() { + return metastoreConfig.enableMetastore(); + } + public static class Builder { protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java index 922371c4a0f45..1182c45c72479 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java @@ -121,7 +121,7 @@ public abstract HoodieData updateLocation( public abstract boolean isImplicitWithStorage(); /** - * If the `getCustomizedPartitioner` returns a partitioner, it has to be true. + * To indicate if a operation type requires location tagging before writing */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public boolean requiresTagging(WriteOperationType operationType) { @@ -143,4 +143,8 @@ public void close() { public enum IndexType { HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE, BUCKET, FLINK_STATE } + + public enum BucketIndexEngineType { + SIMPLE, CONSISTENT_HASHING + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index b714c50334b4f..9b3dc8df0098a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -83,7 +83,7 @@ public static List getLatestBaseFilesForPartition( public static List> getLatestBaseFilesForAllPartitions(final List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { - context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions"); + context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions: " + hoodieTable.getConfig().getTableName()); return context.flatMap(partitions, partitionPath -> { List> filteredFiles = getLatestBaseFilesForPartition(partitionPath, hoodieTable).stream() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index aeaf78672680d..6545c642c4ccb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -167,7 +167,7 @@ List> loadColumnRangesFromFiles( .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId())) .collect(toList()); - context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)"); + context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on): " + config.getTableName()); return context.map(partitionPathFileIDList, pf -> { try { HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf); @@ -209,7 +209,7 @@ private List> getFileInfoForLatestBaseFiles( protected List> loadColumnRangesFromMetaIndex( List partitions, final HoodieEngineContext context, final HoodieTable hoodieTable) { // also obtain file ranges, if range pruning is enabled - context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices"); + context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices: " + config.getTableName()); final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); return context.flatMap(partitions, partitionName -> { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java index 1a07c4063f358..48ccce1d1740c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; +import java.io.Serializable; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -29,8 +30,8 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -public class BucketIdentifier { - // compatible with the spark bucket name +public class BucketIdentifier implements Serializable { + // Compatible with the spark bucket name private static final Pattern BUCKET_NAME = Pattern.compile(".*_(\\d+)(?:\\..*)?$"); public static int getBucketId(HoodieRecord record, String indexKeyFields, int numBuckets) { @@ -38,27 +39,41 @@ public static int getBucketId(HoodieRecord record, String indexKeyFields, int nu } public static int getBucketId(HoodieKey hoodieKey, String indexKeyFields, int numBuckets) { - return getBucketId(hoodieKey.getRecordKey(), indexKeyFields, numBuckets); + return (getHashKeys(hoodieKey, indexKeyFields).hashCode() & Integer.MAX_VALUE) % numBuckets; + } + + public static int getBucketId(HoodieKey hoodieKey, List indexKeyFields, int numBuckets) { + return (getHashKeys(hoodieKey.getRecordKey(), indexKeyFields).hashCode() & Integer.MAX_VALUE) % numBuckets; } public static int getBucketId(String recordKey, String indexKeyFields, int numBuckets) { - List hashKeyFields; - if (!recordKey.contains(":")) { - hashKeyFields = Collections.singletonList(recordKey); - } else { - Map recordKeyPairs = Arrays.stream(recordKey.split(",")) - .map(p -> p.split(":")) - .collect(Collectors.toMap(p -> p[0], p -> p[1])); - hashKeyFields = Arrays.stream(indexKeyFields.split(",")) - .map(f -> recordKeyPairs.get(f)) - .collect(Collectors.toList()); - } - return (hashKeyFields.hashCode() & Integer.MAX_VALUE) % numBuckets; + return getBucketId(getHashKeys(recordKey, indexKeyFields), numBuckets); } - // only for test public static int getBucketId(List hashKeyFields, int numBuckets) { - return hashKeyFields.hashCode() % numBuckets; + return (hashKeyFields.hashCode() & Integer.MAX_VALUE) % numBuckets; + } + + public static List getHashKeys(HoodieKey hoodieKey, String indexKeyFields) { + return getHashKeys(hoodieKey.getRecordKey(), indexKeyFields); + } + + protected static List getHashKeys(String recordKey, String indexKeyFields) { + return !recordKey.contains(":") ? Collections.singletonList(recordKey) : + getHashKeysUsingIndexFields(recordKey, Arrays.asList(indexKeyFields.split(","))); + } + + protected static List getHashKeys(String recordKey, List indexKeyFields) { + return !recordKey.contains(":") ? Collections.singletonList(recordKey) : + getHashKeysUsingIndexFields(recordKey, indexKeyFields); + } + + private static List getHashKeysUsingIndexFields(String recordKey, List indexKeyFields) { + Map recordKeyPairs = Arrays.stream(recordKey.split(",")) + .map(p -> p.split(":")) + .collect(Collectors.toMap(p -> p[0], p -> p[1])); + return indexKeyFields.stream() + .map(recordKeyPairs::get).collect(Collectors.toList()); } public static String partitionBucketIdStr(String partition, int bucketId) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.java new file mode 100644 index 0000000000000..4955087333a25 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIndexLocationMapper.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.index.bucket; + +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.Option; + +import java.io.Serializable; + +public interface BucketIndexLocationMapper extends Serializable { + + /** + * Get record location given hoodie key and partition path + */ + Option getRecordLocation(HoodieKey key, String partitionPath); + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java new file mode 100644 index 0000000000000..c44a8a6ccfb0c --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/ConsistentBucketIdentifier.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bucket; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.util.hash.HashID; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +public class ConsistentBucketIdentifier extends BucketIdentifier { + + /** + * Hashing metadata of a partition + */ + private final HoodieConsistentHashingMetadata metadata; + /** + * In-memory structure to speed up ring mapping (hashing value -> hashing node) + */ + private final TreeMap ring; + /** + * Mapping from fileId -> hashing node + */ + private final Map fileIdToBucket; + + public ConsistentBucketIdentifier(HoodieConsistentHashingMetadata metadata) { + this.metadata = metadata; + this.fileIdToBucket = new HashMap<>(); + this.ring = new TreeMap<>(); + initialize(); + } + + public Collection getNodes() { + return ring.values(); + } + + public HoodieConsistentHashingMetadata getMetadata() { + return metadata; + } + + public int getNumBuckets() { + return ring.size(); + } + + /** + * Get bucket of the given file group + * + * @param fileId the file group id. NOTE: not filePfx (i.e., uuid) + */ + public ConsistentHashingNode getBucketByFileId(String fileId) { + return fileIdToBucket.get(fileId); + } + + public ConsistentHashingNode getBucket(HoodieKey hoodieKey, List indexKeyFields) { + return getBucket(getHashKeys(hoodieKey.getRecordKey(), indexKeyFields)); + } + + protected ConsistentHashingNode getBucket(List hashKeys) { + int hashValue = HashID.getXXHash32(String.join("", hashKeys), 0); + return getBucket(hashValue & HoodieConsistentHashingMetadata.HASH_VALUE_MASK); + } + + protected ConsistentHashingNode getBucket(int hashValue) { + SortedMap tailMap = ring.tailMap(hashValue); + return tailMap.isEmpty() ? ring.firstEntry().getValue() : tailMap.get(tailMap.firstKey()); + } + + /** + * Initialize necessary data structure to facilitate bucket identifying. + * Specifically, we construct: + * - An in-memory tree (ring) to speed up range mapping searching. + * - A hash table (fileIdToBucket) to allow lookup of bucket using fileId. + */ + private void initialize() { + for (ConsistentHashingNode p : metadata.getNodes()) { + ring.put(p.getValue(), p); + // One bucket has only one file group, so append 0 directly + fileIdToBucket.put(FSUtils.createNewFileId(p.getFileIdPrefix(), 0), p); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java index a243eea767856..c3584d234a8e5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieBucketIndex.java @@ -26,9 +26,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; @@ -37,28 +35,31 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.util.HashMap; -import java.util.Map; +import java.util.Arrays; +import java.util.List; /** * Hash indexing mechanism. */ -public class HoodieBucketIndex extends HoodieIndex { +public abstract class HoodieBucketIndex extends HoodieIndex { - private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class); + private static final Logger LOG = LogManager.getLogger(HoodieBucketIndex.class); - private final int numBuckets; + protected final int numBuckets; + protected final List indexKeyFields; public HoodieBucketIndex(HoodieWriteConfig config) { super(config); - numBuckets = config.getBucketIndexNumBuckets(); - LOG.info("use bucket index, numBuckets=" + numBuckets); + + this.numBuckets = config.getBucketIndexNumBuckets(); + this.indexKeyFields = Arrays.asList(config.getBucketIndexHashField().split(",")); + LOG.info("Use bucket index, numBuckets = " + numBuckets + ", indexFields: " + indexKeyFields); } @Override public HoodieData updateLocation(HoodieData writeStatuses, - HoodieEngineContext context, - HoodieTable hoodieTable) + HoodieEngineContext context, + HoodieTable hoodieTable) throws HoodieIndexException { return writeStatuses; } @@ -68,62 +69,35 @@ public HoodieData> tagLocation( HoodieData> records, HoodieEngineContext context, HoodieTable hoodieTable) throws HoodieIndexException { - HoodieData> taggedRecords = records.mapPartitions(recordIter -> { - // partitionPath -> bucketId -> fileInfo - Map>> partitionPathFileIDList = new HashMap<>(); - return new LazyIterableIterator, HoodieRecord>(recordIter) { - - @Override - protected void start() { - - } - - @Override - protected HoodieRecord computeNext() { - HoodieRecord record = recordIter.next(); - int bucketId = BucketIdentifier.getBucketId(record, config.getBucketIndexHashField(), numBuckets); - String partitionPath = record.getPartitionPath(); - if (!partitionPathFileIDList.containsKey(partitionPath)) { - partitionPathFileIDList.put(partitionPath, loadPartitionBucketIdFileIdMapping(hoodieTable, partitionPath)); - } - if (partitionPathFileIDList.get(partitionPath).containsKey(bucketId)) { - Pair fileInfo = partitionPathFileIDList.get(partitionPath).get(bucketId); - return HoodieIndexUtils.getTaggedRecord(record, Option.of( - new HoodieRecordLocation(fileInfo.getRight(), fileInfo.getLeft()) - )); + // Initialize necessary information before tagging. e.g., hashing metadata + List partitions = records.map(HoodieRecord::getPartitionPath).distinct().collectAsList(); + LOG.info("Initializing hashing metadata for partitions: " + partitions); + BucketIndexLocationMapper mapper = getLocationMapper(hoodieTable, partitions); + + return records.mapPartitions(iterator -> + new LazyIterableIterator, HoodieRecord>(iterator) { + @Override + protected HoodieRecord computeNext() { + // TODO maybe batch the operation to improve performance + HoodieRecord record = inputItr.next(); + Option loc = mapper.getRecordLocation(record.getKey(), record.getPartitionPath()); + return HoodieIndexUtils.getTaggedRecord(record, loc); } - return record; - } - - @Override - protected void end() { - } - }; - }, true); - return taggedRecords; + ); } - private Map> loadPartitionBucketIdFileIdMapping( - HoodieTable hoodieTable, - String partition) { - // bucketId -> fileIds - Map> fileIDList = new HashMap<>(); - HoodieIndexUtils - .getLatestBaseFilesForPartition(partition, hoodieTable) - .forEach(file -> { - String fileId = file.getFileId(); - String commitTime = file.getCommitTime(); - int bucketId = BucketIdentifier.bucketIdFromFileId(fileId); - if (!fileIDList.containsKey(bucketId)) { - fileIDList.put(bucketId, Pair.of(fileId, commitTime)); - } else { - // check if bucket data is valid - throw new HoodieIOException("Find multiple files at partition path=" - + partition + " belongs to the same bucket id = " + bucketId); - } - }); - return fileIDList; + @Override + public boolean requiresTagging(WriteOperationType operationType) { + switch (operationType) { + case INSERT: + case INSERT_OVERWRITE: + case UPSERT: + case DELETE: + return true; + default: + return false; + } } @Override @@ -138,7 +112,7 @@ public boolean isGlobal() { @Override public boolean canIndexLogFiles() { - return false; + return true; } @Override @@ -146,19 +120,12 @@ public boolean isImplicitWithStorage() { return true; } - @Override - public boolean requiresTagging(WriteOperationType operationType) { - switch (operationType) { - case INSERT: - case INSERT_OVERWRITE: - case UPSERT: - return true; - default: - return false; - } - } - public int getNumBuckets() { return numBuckets; } + + /** + * Get a location mapper for the given table & partitionPath + */ + protected abstract BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java new file mode 100644 index 0000000000000..92ac4f69b2c42 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bucket; + +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Simple bucket index implementation, with fixed bucket number. + */ +public class HoodieSimpleBucketIndex extends HoodieBucketIndex { + + private static final Logger LOG = LogManager.getLogger(HoodieSimpleBucketIndex.class); + + public HoodieSimpleBucketIndex(HoodieWriteConfig config) { + super(config); + } + + private Map loadPartitionBucketIdFileIdMapping( + HoodieTable hoodieTable, + String partition) { + // bucketId -> fileIds + Map bucketIdToFileIdMapping = new HashMap<>(); + hoodieTable.getMetaClient().reloadActiveTimeline(); + HoodieIndexUtils + .getLatestBaseFilesForPartition(partition, hoodieTable) + .forEach(file -> { + String fileId = file.getFileId(); + String commitTime = file.getCommitTime(); + int bucketId = BucketIdentifier.bucketIdFromFileId(fileId); + if (!bucketIdToFileIdMapping.containsKey(bucketId)) { + bucketIdToFileIdMapping.put(bucketId, new HoodieRecordLocation(commitTime, fileId)); + } else { + // Check if bucket data is valid + throw new HoodieIOException("Find multiple files at partition path=" + + partition + " belongs to the same bucket id = " + bucketId); + } + }); + return bucketIdToFileIdMapping; + } + + @Override + public boolean canIndexLogFiles() { + return false; + } + + @Override + protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath) { + return new SimpleBucketIndexLocationMapper(table, partitionPath); + } + + public class SimpleBucketIndexLocationMapper implements BucketIndexLocationMapper { + + /** + * Mapping from partitionPath -> bucketId -> fileInfo + */ + private final Map> partitionPathFileIDList; + + public SimpleBucketIndexLocationMapper(HoodieTable table, List partitions) { + partitionPathFileIDList = partitions.stream().collect(Collectors.toMap(p -> p, p -> loadPartitionBucketIdFileIdMapping(table, p))); + } + + @Override + public Option getRecordLocation(HoodieKey key, String partitionPath) { + int bucketId = BucketIdentifier.getBucketId(key, indexKeyFields, numBuckets); + Map bucketIdToFileIdMapping = partitionPathFileIDList.get(partitionPath); + return Option.ofNullable(bucketIdToFileIdMapping.getOrDefault(bucketId, null)); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java index 022f600b5e078..ca245e0c391ba 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java @@ -35,6 +35,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; import java.util.Collections; import java.util.Iterator; @@ -66,6 +68,7 @@ * Users should ensure there are no duplicates when "insert" operation is used and if the respective config is enabled. So, above scenario should not * happen and every batch should have new records to be inserted. Above example is for illustration purposes only. */ +@NotThreadSafe public class HoodieConcatHandle extends HoodieMergeHandle { private static final Logger LOG = LogManager.getLogger(HoodieConcatHandle.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 41d583668a933..43a8c12324136 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -42,12 +42,15 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; +@NotThreadSafe public class HoodieCreateHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 3363571ddf0cb..b999cc6906406 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -54,6 +54,8 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; import java.util.Collections; import java.util.HashSet; @@ -91,6 +93,7 @@ * *

*/ +@NotThreadSafe public class HoodieMergeHandle extends HoodieWriteHandle { private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class); @@ -262,6 +265,9 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord ol if (oldRecord != record) { // the incoming record is chosen isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); + } else { + // the incoming record is dropped + return false; } } return writeRecord(hoodieRecord, indexedRecord, isDelete); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java index d6c1d1be40f36..7dce31a4c349b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java @@ -32,6 +32,8 @@ import org.apache.avro.generic.GenericRecord; +import javax.annotation.concurrent.NotThreadSafe; + import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -45,6 +47,7 @@ * The implementation performs a merge-sort by comparing the key of the record being written to the list of * keys in newRecordKeys (sorted in-memory). */ +@NotThreadSafe public class HoodieSortedMergeHandle extends HoodieMergeHandle { private final Queue newRecordKeysSorted = new PriorityQueue<>(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java index 9ab44d0f62f1b..ebbc7a5c28ea1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieUnboundedCreateHandle.java @@ -28,11 +28,14 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.concurrent.NotThreadSafe; + /** * A HoodieCreateHandle which writes all data into a single file. *

* Please use this with caution. This can end up creating very large files if not used correctly. */ +@NotThreadSafe public class HoodieUnboundedCreateHandle extends HoodieCreateHandle { private static final Logger LOG = LogManager.getLogger(HoodieUnboundedCreateHandle.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java index 36fae304d77f2..c267b5969d801 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java @@ -19,6 +19,7 @@ package org.apache.hudi.io; import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -32,6 +33,6 @@ public abstract HoodieWriteHandle create(HoodieWriteConfig config, S String partitionPath, String fileIdPrefix, TaskContextSupplier taskContextSupplier); protected String getNextFileId(String idPfx) { - return String.format("%s-%d", idPfx, numFilesWritten++); + return FSUtils.createNewFileId(idPfx, numFilesWritten++); } } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java new file mode 100644 index 0000000000000..6f7940d04d0f2 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.HoodieKey; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; + +/** + * HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides a way to check if + * the current file can take more records with the canWrite() + * + * ATTENTION: HoodieParquetWriter is not thread safe and developer should take care of the order of write and close + */ +@NotThreadSafe +public class HoodieAvroParquetWriter + extends HoodieBaseParquetWriter + implements HoodieFileWriter { + + private final String fileName; + private final String instantTime; + private final TaskContextSupplier taskContextSupplier; + private final boolean populateMetaFields; + private final HoodieAvroWriteSupport writeSupport; + + @SuppressWarnings({"unchecked", "rawtypes"}) + public HoodieAvroParquetWriter(Path file, + HoodieAvroParquetConfig parquetConfig, + String instantTime, + TaskContextSupplier taskContextSupplier, + boolean populateMetaFields) throws IOException { + super(file, (HoodieBaseParquetConfig) parquetConfig); + this.fileName = file.getName(); + this.writeSupport = parquetConfig.getWriteSupport(); + this.instantTime = instantTime; + this.taskContextSupplier = taskContextSupplier; + this.populateMetaFields = populateMetaFields; + } + + @Override + public void writeAvroWithMetadata(HoodieKey key, R avroRecord) throws IOException { + if (populateMetaFields) { + prepRecordWithMetadata(key, avroRecord, instantTime, + taskContextSupplier.getPartitionIdSupplier().get(), getWrittenRecordCount(), fileName); + super.write(avroRecord); + writeSupport.add(key.getRecordKey()); + } else { + super.write(avroRecord); + } + } + + @Override + public void writeAvro(String key, IndexedRecord object) throws IOException { + super.write(object); + if (populateMetaFields) { + writeSupport.add(key); + } + } + + @Override + public void close() throws IOException { + super.close(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java new file mode 100644 index 0000000000000..b4aa6de1bd577 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Base class of Hudi's custom {@link ParquetWriter} implementations + * + * @param target type of the object being written into Parquet files (for ex, + * {@code IndexedRecord}, {@code InternalRow}) + */ +public abstract class HoodieBaseParquetWriter extends ParquetWriter { + + private static final int WRITTEN_RECORDS_THRESHOLD_FOR_FILE_SIZE_CHECK = 1000; + + private final AtomicLong writtenRecordCount = new AtomicLong(0); + private final long maxFileSize; + private long lastCachedDataSize = -1; + + public HoodieBaseParquetWriter(Path file, + HoodieBaseParquetConfig> parquetConfig) throws IOException { + super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), + ParquetFileWriter.Mode.CREATE, + parquetConfig.getWriteSupport(), + parquetConfig.getCompressionCodecName(), + parquetConfig.getBlockSize(), + parquetConfig.getPageSize(), + parquetConfig.getPageSize(), + parquetConfig.dictionaryEnabled(), + DEFAULT_IS_VALIDATING_ENABLED, + DEFAULT_WRITER_VERSION, + FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); + + // We cannot accurately measure the snappy compressed output file size. We are choosing a + // conservative 10% + // TODO - compute this compression ratio dynamically by looking at the bytes written to the + // stream and the actual file size reported by HDFS + this.maxFileSize = parquetConfig.getMaxFileSize() + + Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio()); + } + + public boolean canWrite() { + // TODO we can actually do evaluation more accurately: + // if we cache last data size check, since we account for how many records + // were written we can accurately project avg record size, and therefore + // estimate how many more records we can write before cut off + if (lastCachedDataSize == -1 || getWrittenRecordCount() % WRITTEN_RECORDS_THRESHOLD_FOR_FILE_SIZE_CHECK == 0) { + lastCachedDataSize = getDataSize(); + } + return lastCachedDataSize < maxFileSize; + } + + @Override + public void write(R object) throws IOException { + super.write(object); + writtenRecordCount.incrementAndGet(); + } + + protected long getWrittenRecordCount() { + return writtenRecordCount.get(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java index 1d1dd5c9bae6d..cce59d3b6624a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java @@ -18,14 +18,12 @@ package org.apache.hudi.io.storage; -import java.util.concurrent.atomic.AtomicLong; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.avro.generic.IndexedRecord; - import java.io.IOException; public interface HoodieFileWriter { @@ -38,8 +36,8 @@ public interface HoodieFileWriter { void writeAvro(String key, R oldRecord) throws IOException; - default void prepRecordWithMetadata(HoodieKey key, R avroRecord, String instantTime, Integer partitionId, AtomicLong recordIndex, String fileName) { - String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex.getAndIncrement()); + default void prepRecordWithMetadata(HoodieKey key, R avroRecord, String instantTime, Integer partitionId, long recordIndex, String fileName) { + String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, key.getRecordKey(), key.getPartitionPath(), fileName); HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId); return; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java index 7d0c307dbfe53..ffdff25738ed7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java @@ -81,7 +81,7 @@ private static HoodieFi config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled()); - return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields); + return new HoodieAvroParquetWriter<>(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields); } static HoodieFileWriter newHFileFileWriter( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java index 91f79cefa23d2..f065608b29bd5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java @@ -113,7 +113,7 @@ public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig hfileC public void writeAvroWithMetadata(HoodieKey key, R avroRecord) throws IOException { if (populateMetaFields) { prepRecordWithMetadata(key, avroRecord, instantTime, - taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, file.getName()); + taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement(), file.getName()); writeAvro(key.getRecordKey(), avroRecord); } else { writeAvro(key.getRecordKey(), avroRecord); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java index 17d5ead3efb79..a532ac66c987c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java @@ -97,7 +97,7 @@ public HoodieOrcWriter(String instantTime, Path file, HoodieOrcConfig config, Sc @Override public void writeAvroWithMetadata(HoodieKey key, R avroRecord) throws IOException { prepRecordWithMetadata(key, avroRecord, instantTime, - taskContextSupplier.getPartitionIdSupplier().get(), RECORD_INDEX, file.getName()); + taskContextSupplier.getPartitionIdSupplier().get(), RECORD_INDEX.getAndIncrement(), file.getName()); writeAvro(key.getRecordKey(), avroRecord); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java deleted file mode 100644 index 5b3c69ddf943e..0000000000000 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java +++ /dev/null @@ -1,109 +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.io.storage; - -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.avro.HoodieAvroWriteSupport; -import org.apache.hudi.common.engine.TaskContextSupplier; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.parquet.hadoop.ParquetFileWriter; -import org.apache.parquet.hadoop.ParquetWriter; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -/** - * HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides a way to check if - * the current file can take more records with the canWrite() - */ -public class HoodieParquetWriter - extends ParquetWriter implements HoodieFileWriter { - - private static AtomicLong recordIndex = new AtomicLong(1); - - private final Path file; - private final HoodieWrapperFileSystem fs; - private final long maxFileSize; - private final HoodieAvroWriteSupport writeSupport; - private final String instantTime; - private final TaskContextSupplier taskContextSupplier; - private final boolean populateMetaFields; - - public HoodieParquetWriter(String instantTime, - Path file, - HoodieAvroParquetConfig parquetConfig, - Schema schema, - TaskContextSupplier taskContextSupplier, - boolean populateMetaFields) throws IOException { - super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), - ParquetFileWriter.Mode.CREATE, - parquetConfig.getWriteSupport(), - parquetConfig.getCompressionCodecName(), - parquetConfig.getBlockSize(), - parquetConfig.getPageSize(), - parquetConfig.getPageSize(), - parquetConfig.dictionaryEnabled(), - DEFAULT_IS_VALIDATING_ENABLED, - DEFAULT_WRITER_VERSION, - FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); - this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); - this.fs = - (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); - // We cannot accurately measure the snappy compressed output file size. We are choosing a - // conservative 10% - // TODO - compute this compression ratio dynamically by looking at the bytes written to the - // stream and the actual file size reported by HDFS - this.maxFileSize = parquetConfig.getMaxFileSize() - + Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio()); - this.writeSupport = parquetConfig.getWriteSupport(); - this.instantTime = instantTime; - this.taskContextSupplier = taskContextSupplier; - this.populateMetaFields = populateMetaFields; - } - - @Override - public void writeAvroWithMetadata(HoodieKey key, R avroRecord) throws IOException { - if (populateMetaFields) { - prepRecordWithMetadata(key, avroRecord, instantTime, - taskContextSupplier.getPartitionIdSupplier().get(), recordIndex, file.getName()); - super.write(avroRecord); - writeSupport.add(key.getRecordKey()); - } else { - super.write(avroRecord); - } - } - - @Override - public boolean canWrite() { - return getDataSize() < maxFileSize; - } - - @Override - public void writeAvro(String key, IndexedRecord object) throws IOException { - super.write(object); - if (populateMetaFields) { - writeSupport.add(key); - } - } -} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index d080d14a69fad..f5a96fb676131 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -1047,7 +1047,7 @@ protected void cleanIfNecessary(BaseHoodieWriteClient writeClient, String instan private void initialCommit(String createInstantTime, List partitionTypes) { // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); - engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions"); + engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions: " + dataWriteConfig.getTableName()); Map> partitionToRecordsMap = new HashMap<>(); 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 f6f73f633ef5d..807865dae2416 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 @@ -566,7 +566,7 @@ public void finalizeWrite(HoodieEngineContext context, String instantTs, List>> invalidFilesByPartition) { // Now delete partially written files - context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation"); + context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation: " + config.getTableName()); context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> { final FileSystem fileSystem = metaClient.getFs(); LOG.info("Deleting invalid data files=" + partitionWithFileList); @@ -642,7 +642,7 @@ protected void reconcileAgainstMarkers(HoodieEngineContext context, } // Now delete partially written files - context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files"); + context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files: " + config.getTableName()); deleteInvalidFilesByPartitions(context, invalidPathsByPartition); // Now ensure the deleted files disappear @@ -665,7 +665,7 @@ protected void reconcileAgainstMarkers(HoodieEngineContext context, */ private void waitForAllFiles(HoodieEngineContext context, Map>> groupByPartition, FileVisibility visibility) { // This will either ensure all files to be deleted are present. - context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear"); + context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear: " + config.getTableName()); boolean checkPassed = context.map(new ArrayList<>(groupByPartition.entrySet()), partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(), partitionWithFileList.getValue().stream(), visibility), config.getFinalizeWriteParallelism()) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index 2bb277b05b4f8..30ed27b39b77a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -132,7 +132,7 @@ List clean(HoodieEngineContext context, HoodieCleanerPlan clean config.getCleanerParallelism()); LOG.info("Using cleanerParallelism: " + cleanerParallelism); - context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions"); + context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions: " + config.getTableName()); Stream> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() 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 fb2df582bfe15..d8e51bcd1643e 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 @@ -96,7 +96,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { try { CleanPlanner planner = new CleanPlanner<>(context, table, config); Option earliestInstant = planner.getEarliestCommitToRetain(); - context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned"); + context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned: " + config.getTableName()); List partitionsToClean = planner.getPartitionPathsToClean(earliestInstant); if (partitionsToClean.isEmpty()) { @@ -107,7 +107,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism()); LOG.info("Using cleanerParallelism: " + cleanerParallelism); - context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned"); + context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned: " + config.getTableName()); Map>> cleanOpsWithPartitionMeta = context .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index fb07d35928d7c..31c8bbd6d30d2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -94,7 +94,7 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(table.getMetaClient()); this.pendingInflightAndRequestedInstants.remove(instantTime); - if (table.getStorageLayout().doesNotSupport(operationType)) { + if (!table.getStorageLayout().writeOperationSupported(operationType)) { throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName() + " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java index 6d5372b47297d..846afec7c1db3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java @@ -49,7 +49,7 @@ public HoodieWriteMetadata write(String instantTime, I taggedRecords = dedupedRecords; if (table.getIndex().requiresTagging(operationType)) { // perform index loop up to get existing location of records - context.setJobStatus(this.getClass().getSimpleName(), "Tagging"); + context.setJobStatus(this.getClass().getSimpleName(), "Tagging: " + table.getConfig().getTableName()); taggedRecords = tag(dedupedRecords, context, table); } Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 04dd29c63c5b4..3e2d8abdd7466 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -148,13 +148,16 @@ public void runMerge(HoodieTable>, HoodieData compact( .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); LOG.info("Compactor compacting " + operations + " files"); - context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices"); + context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices: " + config.getTableName()); TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier(); return context.parallelize(operations).map(operation -> compact( compactionHandler, metaClient, config, operation, compactionInstantTime, taskContextSupplier)) @@ -288,7 +288,7 @@ HoodieCompactionPlan generateCompactionPlan( SliceView fileSystemView = hoodieTable.getSliceView(); LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions"); - context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact"); + context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact: " + config.getTableName()); List operations = context.flatMap(partitionPaths, partitionPath -> fileSystemView .getLatestFileSlices(partitionPath) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java index 24c0dbc80ed80..5c184e77dfaa2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java @@ -88,7 +88,7 @@ public HoodieWriteMetadata> execute() { context, compactionPlan, table, configCopy, instantTime, compactionHandler); compactor.maybePersist(statuses, config); - context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata"); + context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata: " + config.getTableName()); List updateStatusMap = statuses.map(WriteStatus::getStat).collectAsList(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); for (HoodieWriteStat stat : updateStatusMap) { 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 d3cc5660bc70a..05fb7c0c92d1d 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 @@ -119,7 +119,7 @@ private HoodieCompactionPlan scheduleCompaction() { .collect(Collectors.toSet()); // exclude files in pending clustering from compaction. fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); - context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan"); + context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan: " + config.getTableName()); return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java index 8475afe16eea0..8d5e767307d78 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java @@ -72,7 +72,7 @@ public BaseRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteConfig co public List performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1); - context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions"); + context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions: " + config.getTableName()); // If not for conversion to HoodieRollbackInternalRequests, code fails. Using avro model (HoodieRollbackRequest) within spark.parallelize // is failing with com.esotericsoftware.kryo.KryoException // stack trace: https://gist.github.com/nsivabalan/b6359e7d5038484f8043506c8bc9e1c8 @@ -88,7 +88,7 @@ public List performRollback(HoodieEngineContext context, Hoo public List collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List rollbackRequests) { int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1); - context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade"); + context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade: " + config.getTableName()); // If not for conversion to HoodieRollbackInternalRequests, code fails. Using avro model (HoodieRollbackRequest) within spark.parallelize // is failing with com.esotericsoftware.kryo.KryoException // stack trace: https://gist.github.com/nsivabalan/b6359e7d5038484f8043506c8bc9e1c8 diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index e3159abad8de7..aa9e0b6583a24 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -88,7 +88,7 @@ public List getRollbackRequests(HoodieInstant instantToRo FSUtils.getAllPartitionPaths(context, table.getMetaClient().getBasePath(), false, false); int numPartitions = Math.max(Math.min(partitionPaths.size(), config.getRollbackParallelism()), 1); - context.setJobStatus(this.getClass().getSimpleName(), "Creating Listing Rollback Plan"); + context.setJobStatus(this.getClass().getSimpleName(), "Creating Listing Rollback Plan: " + config.getTableName()); HoodieTableType tableType = table.getMetaClient().getTableType(); String baseFileExtension = getBaseFileExtension(metaClient); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index 134b238852cd3..7f408c1b8d24a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -84,7 +84,7 @@ public HoodieSavepointMetadata execute() { ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained), "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained); - context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime); + context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime + " " + table.getConfig().getTableName()); List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); Map> latestFilesMap = context.mapToPair(partitions, partitionPath -> { // Scan all partitions files with this commit time diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java index 3dacf1e1302c5..07428dd936469 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkers.java @@ -84,7 +84,7 @@ public Option createIfNotExists(String partitionPath, String dataFileName, */ public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) { try { - context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory"); + context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory: " + basePath); deleteMarkerDir(context, parallelism); } catch (Exception e) { LOG.warn("Error deleting marker directory for instant " + instantTime, e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java new file mode 100644 index 0000000000000..0ed2b9c939a7b --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieConsistentBucketLayout.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.storage; + +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; + +import java.util.Set; + +/** + * Storage layout when using consistent hashing bucket index. + */ +public class HoodieConsistentBucketLayout extends HoodieStorageLayout { + public static final Set SUPPORTED_OPERATIONS = CollectionUtils.createImmutableSet( + WriteOperationType.INSERT, + WriteOperationType.INSERT_PREPPED, + WriteOperationType.UPSERT, + WriteOperationType.UPSERT_PREPPED, + WriteOperationType.INSERT_OVERWRITE, + WriteOperationType.DELETE, + WriteOperationType.COMPACT, + WriteOperationType.DELETE_PARTITION + ); + + public HoodieConsistentBucketLayout(HoodieWriteConfig config) { + super(config); + } + + /** + * Bucketing controls the number of file groups directly. + */ + @Override + public boolean determinesNumFileGroups() { + return true; + } + + /** + * Consistent hashing will tag all incoming records, so we could go ahead reusing an existing Partitioner + */ + @Override + public Option layoutPartitionerClass() { + return Option.empty(); + } + + @Override + public boolean writeOperationSupported(WriteOperationType operationType) { + return SUPPORTED_OPERATIONS.contains(operationType); + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java index 09d20707a4c85..28fe37c9b8fe0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieDefaultLayout.java @@ -31,15 +31,18 @@ public HoodieDefaultLayout(HoodieWriteConfig config) { super(config); } + @Override public boolean determinesNumFileGroups() { return false; } + @Override public Option layoutPartitionerClass() { return Option.empty(); } - public boolean doesNotSupport(WriteOperationType operationType) { - return false; + @Override + public boolean writeOperationSupported(WriteOperationType operationType) { + return true; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java index e86d253df4bfa..e78c15b3a4b22 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieLayoutFactory.java @@ -30,7 +30,14 @@ public static HoodieStorageLayout createLayout(HoodieWriteConfig config) { case DEFAULT: return new HoodieDefaultLayout(config); case BUCKET: - return new HoodieBucketLayout(config); + switch (config.getBucketIndexEngineType()) { + case SIMPLE: + return new HoodieSimpleBucketLayout(config); + case CONSISTENT_HASHING: + return new HoodieConsistentBucketLayout(config); + default: + throw new HoodieNotSupportedException("Unknown bucket index engine type: " + config.getBucketIndexEngineType()); + } default: throw new HoodieNotSupportedException("Unknown layout type, set " + config.getLayoutType()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieSimpleBucketLayout.java similarity index 71% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieSimpleBucketLayout.java index deefcfe6a621e..be048a23b058c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieBucketLayout.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieSimpleBucketLayout.java @@ -19,31 +19,30 @@ package org.apache.hudi.table.storage; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieLayoutConfig; import org.apache.hudi.config.HoodieWriteConfig; -import java.util.HashSet; import java.util.Set; /** * Storage layout when using bucket index. Data distribution and files organization are in a specific way. */ -public class HoodieBucketLayout extends HoodieStorageLayout { +public class HoodieSimpleBucketLayout extends HoodieStorageLayout { - public static final Set SUPPORTED_OPERATIONS = new HashSet() {{ - add(WriteOperationType.INSERT); - add(WriteOperationType.INSERT_PREPPED); - add(WriteOperationType.UPSERT); - add(WriteOperationType.UPSERT_PREPPED); - add(WriteOperationType.INSERT_OVERWRITE); - add(WriteOperationType.DELETE); - add(WriteOperationType.COMPACT); - add(WriteOperationType.DELETE_PARTITION); - } - }; + public static final Set SUPPORTED_OPERATIONS = CollectionUtils.createImmutableSet( + WriteOperationType.INSERT, + WriteOperationType.INSERT_PREPPED, + WriteOperationType.UPSERT, + WriteOperationType.UPSERT_PREPPED, + WriteOperationType.INSERT_OVERWRITE, + WriteOperationType.DELETE, + WriteOperationType.COMPACT, + WriteOperationType.DELETE_PARTITION + ); - public HoodieBucketLayout(HoodieWriteConfig config) { + public HoodieSimpleBucketLayout(HoodieWriteConfig config) { super(config); } @@ -55,6 +54,7 @@ public boolean determinesNumFileGroups() { return true; } + @Override public Option layoutPartitionerClass() { return config.contains(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME) ? Option.of(config.getString(HoodieLayoutConfig.LAYOUT_PARTITIONER_CLASS_NAME.key())) @@ -62,7 +62,7 @@ public Option layoutPartitionerClass() { } @Override - public boolean doesNotSupport(WriteOperationType operationType) { - return !SUPPORTED_OPERATIONS.contains(operationType); + public boolean writeOperationSupported(WriteOperationType operationType) { + return SUPPORTED_OPERATIONS.contains(operationType); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java index a0a4eab46304f..36be1a8bef6a8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/storage/HoodieStorageLayout.java @@ -48,7 +48,7 @@ public HoodieStorageLayout(HoodieWriteConfig config) { /** * Determines if the operation is supported by the layout. */ - public abstract boolean doesNotSupport(WriteOperationType operationType); + public abstract boolean writeOperationSupported(WriteOperationType operationType); public enum LayoutType { DEFAULT, BUCKET diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java new file mode 100644 index 0000000000000..31f33890ad318 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bucket; + +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.keygen.KeyGenUtils; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +public class TestBucketIdentifier { + + public static final String NESTED_COL_SCHEMA = "{\"type\":\"record\", \"name\":\"nested_col\",\"fields\": [" + + "{\"name\": \"prop1\",\"type\": \"string\"},{\"name\": \"prop2\", \"type\": \"long\"}]}"; + public static final String EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"testrec\",\"fields\": [ " + + "{\"name\": \"timestamp\",\"type\": \"long\"},{\"name\": \"_row_key\", \"type\": \"string\"}," + + "{\"name\": \"ts_ms\", \"type\": \"string\"}," + + "{\"name\": \"pii_col\", \"type\": \"string\"}," + + "{\"name\": \"nested_col\",\"type\": " + + NESTED_COL_SCHEMA + "}" + + "]}"; + + public static GenericRecord getRecord() { + return getRecord(getNestedColRecord("val1", 10L)); + } + + public static GenericRecord getNestedColRecord(String prop1Value, Long prop2Value) { + GenericRecord nestedColRecord = new GenericData.Record(new Schema.Parser().parse(NESTED_COL_SCHEMA)); + nestedColRecord.put("prop1", prop1Value); + nestedColRecord.put("prop2", prop2Value); + return nestedColRecord; + } + + public static GenericRecord getRecord(GenericRecord nestedColRecord) { + GenericRecord record = new GenericData.Record(new Schema.Parser().parse(EXAMPLE_SCHEMA)); + record.put("timestamp", 4357686L); + record.put("_row_key", "key1"); + record.put("ts_ms", "2020-03-21"); + record.put("pii_col", "pi"); + record.put("nested_col", nestedColRecord); + return record; + } + + @Test + public void testBucketFileId() { + int[] ids = {0, 4, 8, 16, 32, 64, 128, 256, 512, 1000, 1024, 4096, 10000, 100000}; + for (int id : ids) { + String bucketIdStr = BucketIdentifier.bucketIdStr(id); + String fileId = BucketIdentifier.newBucketFileIdPrefix(bucketIdStr); + assert BucketIdentifier.bucketIdFromFileId(fileId) == id; + } + } + + @Test + public void testBucketIdWithSimpleRecordKey() { + String recordKeyField = "_row_key"; + String indexKeyField = "_row_key"; + GenericRecord record = getRecord(); + HoodieRecord hoodieRecord = new HoodieAvroRecord( + new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); + int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); + assert bucketId == BucketIdentifier.getBucketId( + Arrays.asList(record.get(indexKeyField).toString()), 8); + } + + @Test + public void testBucketIdWithComplexRecordKey() { + List recordKeyField = Arrays.asList("_row_key", "ts_ms"); + String indexKeyField = "_row_key"; + GenericRecord record = getRecord(); + HoodieRecord hoodieRecord = new HoodieAvroRecord( + new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); + int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); + assert bucketId == BucketIdentifier.getBucketId( + Arrays.asList(record.get(indexKeyField).toString()), 8); + } + + @Test + public void testGetHashKeys() { + BucketIdentifier identifier = new BucketIdentifier(); + List keys = identifier.getHashKeys(new HoodieKey("abc", "partition"), ""); + Assertions.assertEquals(1, keys.size()); + Assertions.assertEquals("abc", keys.get(0)); + + keys = identifier.getHashKeys(new HoodieKey("f1:abc", "partition"), "f1"); + Assertions.assertEquals(1, keys.size()); + Assertions.assertEquals("abc", keys.get(0)); + + keys = identifier.getHashKeys(new HoodieKey("f1:abc,f2:bcd", "partition"), "f2"); + Assertions.assertEquals(1, keys.size()); + Assertions.assertEquals("bcd", keys.get(0)); + + keys = identifier.getHashKeys(new HoodieKey("f1:abc,f2:bcd", "partition"), "f1,f2"); + Assertions.assertEquals(2, keys.size()); + Assertions.assertEquals("abc", keys.get(0)); + Assertions.assertEquals("bcd", keys.get(1)); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java new file mode 100644 index 0000000000000..3ffe6ded188b8 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/index/bucket/TestConsistentBucketIdIdentifier.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bucket; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASH_VALUE_MASK; + +/** + * Unit test of consistent bucket identifier + */ +public class TestConsistentBucketIdIdentifier { + + @Test + public void testGetBucket() { + List nodes = Arrays.asList( + new ConsistentHashingNode(100, "0"), + new ConsistentHashingNode(0x2fffffff, "1"), + new ConsistentHashingNode(0x4fffffff, "2")); + HoodieConsistentHashingMetadata meta = new HoodieConsistentHashingMetadata((short) 0, "", "", 3, 0, nodes); + ConsistentBucketIdentifier identifier = new ConsistentBucketIdentifier(meta); + + Assertions.assertEquals(3, identifier.getNumBuckets()); + + // Get bucket by hash keys + Assertions.assertEquals(nodes.get(2), identifier.getBucket(Arrays.asList("Hudi"))); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index"))); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("consistent_hashing"))); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(Arrays.asList("bucket_index", "consistent_hashing"))); + int[] ref1 = {2, 2, 1, 1, 0, 1, 1, 1, 0, 1}; + int[] ref2 = {1, 0, 1, 0, 1, 1, 1, 0, 1, 2}; + for (int i = 0; i < 10; ++i) { + Assertions.assertEquals(nodes.get(ref1[i]), identifier.getBucket(Arrays.asList(Integer.toString(i)))); + Assertions.assertEquals(nodes.get(ref2[i]), identifier.getBucket(Arrays.asList(Integer.toString(i), Integer.toString(i + 1)))); + } + + // Get bucket by hash value + Assertions.assertEquals(nodes.get(0), identifier.getBucket(0)); + Assertions.assertEquals(nodes.get(0), identifier.getBucket(50)); + Assertions.assertEquals(nodes.get(0), identifier.getBucket(100)); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(101)); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(0x1fffffff)); + Assertions.assertEquals(nodes.get(1), identifier.getBucket(0x2fffffff)); + Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x40000000)); + Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x40000001)); + Assertions.assertEquals(nodes.get(2), identifier.getBucket(0x4fffffff)); + Assertions.assertEquals(nodes.get(0), identifier.getBucket(0x50000000)); + Assertions.assertEquals(nodes.get(0), identifier.getBucket(HASH_VALUE_MASK)); + + // Get bucket by file id + Assertions.assertEquals(nodes.get(0), identifier.getBucketByFileId(FSUtils.createNewFileId("0", 0))); + Assertions.assertEquals(nodes.get(1), identifier.getBucketByFileId(FSUtils.createNewFileId("1", 0))); + Assertions.assertEquals(nodes.get(2), identifier.getBucketByFileId(FSUtils.createNewFileId("2", 0))); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java index 007ad290aadd9..6b847d4960fb6 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java @@ -41,7 +41,7 @@ import org.apache.hudi.io.storage.HoodieAvroParquetConfig; import org.apache.hudi.io.storage.HoodieOrcConfig; import org.apache.hudi.io.storage.HoodieOrcWriter; -import org.apache.hudi.io.storage.HoodieParquetWriter; +import org.apache.hudi.io.storage.HoodieAvroParquetWriter; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.avro.Schema; @@ -113,10 +113,9 @@ public Path withInserts(String partition, String fileId, List reco HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue())); - try (HoodieParquetWriter writer = new HoodieParquetWriter( - currentInstantTime, - new Path(Paths.get(basePath, partition, fileName).toString()), - config, schema, contextSupplier, populateMetaFields)) { + try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter<>( + new Path(Paths.get(basePath, partition, fileName).toString()), config, currentInstantTime, + contextSupplier, populateMetaFields)) { int seqId = 1; for (HoodieRecord record : records) { GenericRecord avroRecord = (GenericRecord) ((HoodieRecordPayload) record.getData()).getInsertValue(schema).get(); 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 271ba95d941e8..2d23c3afb7f14 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 @@ -356,9 +356,9 @@ public void completeCompaction( HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime) { - this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName()); List writeStats = metadata.getWriteStats(); - final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime); + final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); try { this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); finalizeWrite(table, compactionCommitTime, writeStats); @@ -372,6 +372,9 @@ public void completeCompaction( } finally { this.txnManager.endTransaction(Option.of(compactionInstant)); } + WriteMarkersFactory + .get(config.getMarkersType(), table, compactionCommitTime) + .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); try { @@ -404,14 +407,20 @@ protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + // do nothing. + // flink executes the upgrade/downgrade once when initializing the first instant on start up, + // no need to execute the upgrade/downgrade on each write in streaming. + } + /** * Upgrade downgrade the Hoodie table. * *

This action should only be executed once for each commit. * The modification of the table properties is not thread safe. */ - public void upgradeDowngrade(String instantTime) { - HoodieTableMetaClient metaClient = createMetaClient(true); + public void upgradeDowngrade(String instantTime, HoodieTableMetaClient metaClient) { new UpgradeDowngrade(metaClient, config, context, FlinkUpgradeDowngradeHelper.getInstance()) .run(HoodieTableVersion.current(), instantTime); } @@ -508,7 +517,7 @@ public Map> getPartitionToReplacedFileIds( List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); if (partitionPaths != null && partitionPaths.size() > 0) { - context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); + context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions: " + config.getTableName()); partitionToExistingFileIds = partitionPaths.stream().parallel() .collect( Collectors.toMap( diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java index 78b3cb1dc61f7..2377ea6ffe51a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -66,6 +66,7 @@ protected List computeNext() { } finally { if (null != bufferedIteratorExecutor) { bufferedIteratorExecutor.shutdownNow(); + bufferedIteratorExecutor.awaitTermination(); } } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 76774e9618d79..222ff78edc9fe 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -138,6 +138,11 @@ protected void commit(String instantTime, Map statuses = preppedRecordList.size() > 0 diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java index 38d4e60f648ec..31312655251ab 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java @@ -102,13 +102,16 @@ public void runMerge(HoodieTable>, List, List } catch (Exception e) { throw new HoodieException(e); } finally { + // HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting + // and executor firstly and then close mergeHandle. if (reader != null) { reader.close(); } - mergeHandle.close(); if (null != wrapper) { wrapper.shutdownNow(); + wrapper.awaitTermination(); } + mergeHandle.close(); } } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java index f91dd5019a275..9821aedc875cd 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java @@ -74,6 +74,7 @@ protected List computeNext() { } finally { if (null != bufferedIteratorExecutor) { bufferedIteratorExecutor.shutdownNow(); + bufferedIteratorExecutor.awaitTermination(); } } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java index 7878d857761ea..46dd30a7cb773 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java @@ -102,13 +102,16 @@ public void runMerge(HoodieTable>, List, List } catch (Exception e) { throw new HoodieException(e); } finally { + // HUDI-2875: mergeHandle is not thread safe, we should totally terminate record inputting + // and executor firstly and then close mergeHandle. if (reader != null) { reader.close(); } - mergeHandle.close(); if (null != wrapper) { wrapper.shutdownNow(); + wrapper.awaitTermination(); } + mergeHandle.close(); } } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java index deaf934cf5d03..fb19259b55591 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java @@ -230,7 +230,7 @@ private Map> getSmallFilesForPartitions(List par } if (partitionPaths != null && partitionPaths.size() > 0) { - context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions"); + context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions: " + config.getTableName()); partitionSmallFilesMap = context.mapToPair(partitionPaths, partitionPath -> new ImmutablePair<>(partitionPath, getSmallFiles(partitionPath)), 0); } 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 518414d614e8f..7b0c4dbdf2a96 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 @@ -379,7 +379,7 @@ public void testFileSizeUpsertRecords() throws Exception { List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB - for (int i = 0; i < 2000; i++) { + for (int i = 0; i < 2050; i++) { String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); @@ -402,7 +402,8 @@ public void testFileSizeUpsertRecords() throws Exception { counts++; } } - assertEquals(5, counts, "If the number of records are more than 1150, then there should be a new file"); + // we check canWrite only once every 1000 records. and so 2 files with 1000 records and 3rd file with 50 records. + assertEquals(3, counts, "If the number of records are more than 1150, then there should be a new file"); } @Test 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 7b0c8bbc8d25c..7f9ec05e3c5eb 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 @@ -117,7 +117,7 @@ protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) { @Override public boolean commit(String instantTime, JavaRDD writeStatuses, Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) { - context.setJobStatus(this.getClass().getSimpleName(), "Committing stats"); + context.setJobStatus(this.getClass().getSimpleName(), "Committing stats: " + config.getTableName()); List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); } @@ -179,7 +179,7 @@ public JavaRDD insert(JavaRDD> records, String inst initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.insert(context,instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> result = table.insert(context, instantTime, HoodieJavaRDD.of(records)); HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); return postWrite(resultRDD, instantTime, table); } @@ -303,9 +303,9 @@ public void commitCompaction(String compactionInstantTime, HoodieCommitMetadata protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime) { - this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); + this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName()); List writeStats = metadata.getWriteStats(); - final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime); + final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); try { this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); finalizeWrite(table, compactionCommitTime, writeStats); @@ -382,7 +382,7 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, + writeStats.stream().filter(s -> s.getTotalWriteErrors() > 0L).map(s -> s.getFileId()).collect(Collectors.joining(","))); } - final HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime); + final HoodieInstant clusteringInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime); try { this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); @@ -393,7 +393,7 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata); table.getActiveTimeline().transitionReplaceInflightToComplete( - HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), + clusteringInstant, Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } catch (Exception e) { throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); 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 66edf607f84dd..0843dfc3c9920 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 @@ -112,6 +112,11 @@ public HoodieData mapPartitions(SerializableFunction, Iterato return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning)); } + @Override + public HoodieData mapPartitions(SerializableFunction, Iterator> func) { + return HoodieJavaRDD.of(rddData.mapPartitions(func::apply)); + } + @Override public HoodieData flatMap(SerializableFunction> func) { return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e))); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index a8a9e49c01c00..df5bd2d3f458c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -95,6 +95,7 @@ protected List computeNext() { } finally { if (null != bufferedIteratorExecutor) { bufferedIteratorExecutor.shutdownNow(); + bufferedIteratorExecutor.awaitTermination(); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java index d1f40dca484c5..4525490c8d168 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java @@ -28,7 +28,8 @@ import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper; -import org.apache.hudi.index.bucket.HoodieBucketIndex; +import org.apache.hudi.index.bucket.HoodieSimpleBucketIndex; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex; @@ -56,8 +57,6 @@ public static HoodieIndex createIndex(HoodieWriteConfig config) { return new SparkHoodieHBaseIndex(config); case INMEMORY: return new HoodieInMemoryHashIndex(config); - case BUCKET: - return new HoodieBucketIndex(config); case BLOOM: return new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); case GLOBAL_BLOOM: @@ -66,6 +65,15 @@ public static HoodieIndex createIndex(HoodieWriteConfig config) { return new HoodieSimpleIndex(config, getKeyGeneratorForSimpleIndex(config)); case GLOBAL_SIMPLE: return new HoodieGlobalSimpleIndex(config, getKeyGeneratorForSimpleIndex(config)); + case BUCKET: + switch (config.getBucketIndexEngineType()) { + case SIMPLE: + return new HoodieSimpleBucketIndex(config); + case CONSISTENT_HASHING: + return new HoodieSparkConsistentBucketIndex(config); + default: + throw new HoodieIndexException("Unknown bucket index engine type: " + config.getBucketIndexEngineType()); + } default: throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType()); } @@ -90,6 +98,8 @@ public static boolean isGlobalIndex(HoodieWriteConfig config) { return false; case GLOBAL_SIMPLE: return true; + case BUCKET: + return false; default: return createIndex(config).isGlobal(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java index 9c2f37d56a509..c9fb895adc401 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java @@ -126,7 +126,7 @@ private Map computeComparisonsPerFileGroup( if (config.getBloomIndexPruneByRanges()) { // we will just try exploding the input and then count to determine comparisons // FIX(vc): Only do sampling here and extrapolate? - context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files"); + context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files: " + config.getTableName()); fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey(); } else { fileToComparisons = new HashMap<>(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java new file mode 100644 index 0000000000000..ca6bf0fc7d990 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bucket; + +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.fs.FSUtils; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.common.model.ConsistentHashingNode; +import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.table.HoodieTable; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * Consistent hashing bucket index implementation, with auto-adjust bucket number. + * NOTE: bucket resizing is triggered by clustering. + */ +public class HoodieSparkConsistentBucketIndex extends HoodieBucketIndex { + + private static final Logger LOG = LogManager.getLogger(HoodieSparkConsistentBucketIndex.class); + + public HoodieSparkConsistentBucketIndex(HoodieWriteConfig config) { + super(config); + } + + @Override + public HoodieData updateLocation(HoodieData writeStatuses, + HoodieEngineContext context, + HoodieTable hoodieTable) + throws HoodieIndexException { + return writeStatuses; + } + + /** + * Do nothing. + * A failed write may create a hashing metadata for a partition. In this case, we still do nothing when rolling back + * the failed write. Because the hashing metadata created by a writer must have 00000000000000 timestamp and can be viewed + * as the initialization of a partition rather than as a part of the failed write. + */ + @Override + public boolean rollbackCommit(String instantTime) { + return true; + } + + @Override + protected BucketIndexLocationMapper getLocationMapper(HoodieTable table, List partitionPath) { + return new ConsistentBucketIndexLocationMapper(table, partitionPath); + } + + /** + * Load hashing metadata of the given partition, if it is not existed, create a new one (also persist it into storage) + * + * @param table hoodie table + * @param partition table partition + * @return Consistent hashing metadata + */ + public HoodieConsistentHashingMetadata loadOrCreateMetadata(HoodieTable table, String partition) { + HoodieConsistentHashingMetadata metadata = loadMetadata(table, partition); + if (metadata != null) { + return metadata; + } + + // There is no metadata, so try to create a new one and save it. + metadata = new HoodieConsistentHashingMetadata(partition, numBuckets); + if (saveMetadata(table, metadata, false)) { + return metadata; + } + + // The creation failed, so try load metadata again. Concurrent creation of metadata should have succeeded. + // Note: the consistent problem of cloud storage is handled internal in the HoodieWrapperFileSystem, i.e., ConsistentGuard + metadata = loadMetadata(table, partition); + ValidationUtils.checkState(metadata != null, "Failed to load or create metadata, partition: " + partition); + return metadata; + } + + /** + * Load hashing metadata of the given partition, if it is not existed, return null + * + * @param table hoodie table + * @param partition table partition + * @return Consistent hashing metadata or null if it does not exist + */ + public static HoodieConsistentHashingMetadata loadMetadata(HoodieTable table, String partition) { + Path metadataPath = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), partition); + + try { + if (!table.getMetaClient().getFs().exists(metadataPath)) { + return null; + } + FileStatus[] metaFiles = table.getMetaClient().getFs().listStatus(metadataPath); + final HoodieTimeline completedCommits = table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(); + Predicate metaFilePredicate = fileStatus -> { + String filename = fileStatus.getPath().getName(); + if (!filename.contains(HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX)) { + return false; + } + String timestamp = HoodieConsistentHashingMetadata.getTimestampFromFile(filename); + return completedCommits.containsInstant(timestamp) || timestamp.equals(HoodieTimeline.INIT_INSTANT_TS); + }; + + // Get a valid hashing metadata with the largest (latest) timestamp + FileStatus metaFile = Arrays.stream(metaFiles).filter(metaFilePredicate) + .max(Comparator.comparing(a -> a.getPath().getName())).orElse(null); + + if (metaFile == null) { + return null; + } + + byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath())); + return HoodieConsistentHashingMetadata.fromBytes(content); + } catch (IOException e) { + LOG.error("Error when loading hashing metadata, partition: " + partition, e); + throw new HoodieIndexException("Error while loading hashing metadata", e); + } + } + + /** + * Save metadata into storage + * + * @param table hoodie table + * @param metadata hashing metadata to be saved + * @param overwrite whether to overwrite existing metadata + * @return true if the metadata is saved successfully + */ + private static boolean saveMetadata(HoodieTable table, HoodieConsistentHashingMetadata metadata, boolean overwrite) { + HoodieWrapperFileSystem fs = table.getMetaClient().getFs(); + Path dir = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), metadata.getPartitionPath()); + Path fullPath = new Path(dir, metadata.getFilename()); + try (FSDataOutputStream fsOut = fs.create(fullPath, overwrite)) { + byte[] bytes = metadata.toBytes(); + fsOut.write(bytes); + fsOut.close(); + return true; + } catch (IOException e) { + LOG.warn("Failed to update bucket metadata: " + metadata, e); + } + return false; + } + + public class ConsistentBucketIndexLocationMapper implements BucketIndexLocationMapper { + + /** + * Mapping from partitionPath -> bucket identifier + */ + private final Map partitionToIdentifier; + + public ConsistentBucketIndexLocationMapper(HoodieTable table, List partitions) { + // TODO maybe parallel + partitionToIdentifier = partitions.stream().collect(Collectors.toMap(p -> p, p -> { + HoodieConsistentHashingMetadata metadata = loadOrCreateMetadata(table, p); + return new ConsistentBucketIdentifier(metadata); + })); + } + + @Override + public Option getRecordLocation(HoodieKey key, String partitionPath) { + ConsistentHashingNode node = partitionToIdentifier.get(partitionPath).getBucket(key, indexKeyFields); + if (!StringUtils.isNullOrEmpty(node.getFileIdPrefix())) { + /** + * Dynamic Bucket Index doesn't need the instant time of the latest file group. + * We add suffix 0 here to the file uuid, following the naming convention, i.e., fileId = [uuid]_[numWrites] + */ + return Option.of(new HoodieRecordLocation(null, FSUtils.createNewFileId(node.getFileIdPrefix(), 0))); + } + + LOG.error("Consistent hashing node has no file group, partition: " + partitionPath + ", meta: " + + partitionToIdentifier.get(partitionPath).getMetadata().getFilename() + ", record_key: " + key.toString()); + throw new HoodieIndexException("Failed to getBucket as hashing node has no file group"); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index fc73a0aed7d70..f841117d5c3a1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -42,7 +42,6 @@ import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionLocation; @@ -60,10 +59,12 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; import org.apache.spark.SparkConf; +import org.apache.spark.SparkFiles; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -72,6 +73,7 @@ import java.io.IOException; import java.io.Serializable; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Date; import java.util.HashMap; @@ -150,9 +152,28 @@ private Connection getHBaseConnection() { } String port = String.valueOf(config.getHbaseZkPort()); hbaseConfig.set("hbase.zookeeper.property.clientPort", port); + try { - return ConnectionFactory.createConnection(hbaseConfig); - } catch (IOException e) { + String authentication = config.getHBaseIndexSecurityAuthentication(); + if (authentication.equals("kerberos")) { + hbaseConfig.set("hbase.security.authentication", "kerberos"); + hbaseConfig.set("hadoop.security.authentication", "kerberos"); + hbaseConfig.set("hbase.security.authorization", "true"); + hbaseConfig.set("hbase.regionserver.kerberos.principal", config.getHBaseIndexRegionserverPrincipal()); + hbaseConfig.set("hbase.master.kerberos.principal", config.getHBaseIndexMasterPrincipal()); + + String principal = config.getHBaseIndexKerberosUserPrincipal(); + String keytab = SparkFiles.get(config.getHBaseIndexKerberosUserKeytab()); + + UserGroupInformation.setConfiguration(hbaseConfig); + UserGroupInformation ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab); + return ugi.doAs((PrivilegedExceptionAction) () -> + (Connection) ConnectionFactory.createConnection(hbaseConfig) + ); + } else { + return ConnectionFactory.createConnection(hbaseConfig); + } + } catch (IOException | InterruptedException e) { throw new HoodieDependentSystemUnavailableException(HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port, e); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java index 7e64d83879f05..5a0a60ea07500 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowParquetWriter.java @@ -19,11 +19,7 @@ package org.apache.hudi.io.storage.row; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; - -import org.apache.parquet.hadoop.ParquetFileWriter; -import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.hudi.io.storage.HoodieBaseParquetWriter; import org.apache.spark.sql.catalyst.InternalRow; import java.io.IOException; @@ -31,32 +27,16 @@ /** * Parquet's impl of {@link HoodieInternalRowFileWriter} to write {@link InternalRow}s. */ -public class HoodieInternalRowParquetWriter extends ParquetWriter +public class HoodieInternalRowParquetWriter extends HoodieBaseParquetWriter implements HoodieInternalRowFileWriter { - private final Path file; - private final HoodieWrapperFileSystem fs; - private final long maxFileSize; private final HoodieRowParquetWriteSupport writeSupport; public HoodieInternalRowParquetWriter(Path file, HoodieRowParquetConfig parquetConfig) throws IOException { - super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), - ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(), - parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), - DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED, - DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); - this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); - this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, - parquetConfig.getHadoopConf())); - this.maxFileSize = parquetConfig.getMaxFileSize() - + Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio()); - this.writeSupport = parquetConfig.getWriteSupport(); - } + super(file, parquetConfig); - @Override - public boolean canWrite() { - return getDataSize() < maxFileSize; + this.writeSupport = parquetConfig.getWriteSupport(); } @Override @@ -69,9 +49,4 @@ public void writeRow(String key, InternalRow row) throws IOException { public void writeRow(InternalRow row) throws IOException { super.write(row); } - - @Override - public void close() throws IOException { - super.close(); - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java index ce3cd6f09768d..4db7eb26e64ba 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowCreateHandle.java @@ -68,8 +68,8 @@ public class HoodieRowCreateHandle implements Serializable { private final HoodieTimer currTimer; public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId, - String instantTime, int taskPartitionId, long taskId, long taskEpochId, - StructType structType) { + String instantTime, int taskPartitionId, long taskId, long taskEpochId, + StructType structType) { this.partitionPath = partitionPath; this.table = table; this.writeConfig = writeConfig; @@ -107,16 +107,15 @@ public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, S /** * Writes an {@link InternalRow} to the underlying HoodieInternalRowFileWriter. Before writing, value for meta columns are computed as required * and wrapped in {@link HoodieInternalRow}. {@link HoodieInternalRow} is what gets written to HoodieInternalRowFileWriter. + * * @param record instance of {@link InternalRow} that needs to be written to the fileWriter. * @throws IOException */ public void write(InternalRow record) throws IOException { try { - String partitionPath = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get( - HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString(); - String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement()); - String recordKey = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get( - HoodieRecord.RECORD_KEY_METADATA_FIELD)).toString(); + final String partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS)); + final String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement()); + final String recordKey = String.valueOf(record.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_POS)); HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(), record); try { @@ -141,6 +140,7 @@ public boolean canWrite() { /** * Closes the {@link HoodieRowCreateHandle} and returns an instance of {@link HoodieInternalWriteStatus} containing the stats and * status of the writes to this handle. + * * @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle. * @throws IOException */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index fe03f60ee816c..0642a85c5f6cd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -18,25 +18,25 @@ package org.apache.hudi.keygen; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.ApiMaturityLevel; import org.apache.hudi.AvroConversionUtils; -import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.PublicAPIMethod; -import org.apache.hudi.client.utils.SparkRowSerDe; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.HoodieKeyException; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; -import scala.Function1; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import scala.Function1; /** * Base class for the built-in key generators. Contains methods structured for @@ -46,13 +46,12 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp private static final String STRUCT_NAME = "hoodieRowTopLevelField"; private static final String NAMESPACE = "hoodieRow"; - private transient Function1 converterFn = null; - private SparkRowSerDe sparkRowSerDe; + private Function1 converterFn = null; + private final AtomicBoolean validatePartitionFields = new AtomicBoolean(false); protected StructType structType; - protected Map> recordKeyPositions = new HashMap<>(); - protected Map> partitionPathPositions = new HashMap<>(); - protected Map> partitionPathDataTypes = null; + protected Map, DataType>> recordKeySchemaInfo = new HashMap<>(); + protected Map, DataType>> partitionPathSchemaInfo = new HashMap<>(); protected BuiltinKeyGenerator(TypedProperties config) { super(config); @@ -60,6 +59,7 @@ protected BuiltinKeyGenerator(TypedProperties config) { /** * Fetch record key from {@link Row}. + * * @param row instance of {@link Row} from which record key is requested. * @return the record key of interest from {@link Row}. */ @@ -74,6 +74,7 @@ public String getRecordKey(Row row) { /** * Fetch partition path from {@link Row}. + * * @param row instance of {@link Row} from which partition path is requested * @return the partition path of interest from {@link Row}. */ @@ -97,87 +98,41 @@ public String getPartitionPath(Row row) { @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public String getPartitionPath(InternalRow internalRow, StructType structType) { try { - initDeserializer(structType); - Row row = sparkRowSerDe.deserializeRow(internalRow); - return getPartitionPath(row); + buildFieldSchemaInfoIfNeeded(structType); + return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(internalRow, getPartitionPathFields(), + hiveStylePartitioning, partitionPathSchemaInfo); } catch (Exception e) { throw new HoodieIOException("Conversion of InternalRow to Row failed with exception " + e); } } - private void initDeserializer(StructType structType) { - if (sparkRowSerDe == null) { - sparkRowSerDe = HoodieSparkUtils.getDeserializer(structType); - } - } - - void buildFieldPositionMapIfNeeded(StructType structType) { + void buildFieldSchemaInfoIfNeeded(StructType structType) { if (this.structType == null) { - // parse simple fields - getRecordKeyFields().stream() - .filter(f -> !(f.contains("."))) - .forEach(f -> { - if (structType.getFieldIndex(f).isDefined()) { - recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get()))); - } else { - throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\""); - } - }); - // parse nested fields - getRecordKeyFields().stream() - .filter(f -> f.contains(".")) - .forEach(f -> recordKeyPositions.put(f, RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, true))); - // parse simple fields + getRecordKeyFields() + .stream().filter(f -> !f.isEmpty()) + .forEach(f -> recordKeySchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, true))); if (getPartitionPathFields() != null) { - getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains("."))) - .forEach(f -> { - if (structType.getFieldIndex(f).isDefined()) { - partitionPathPositions.put(f, - Collections.singletonList((Integer) (structType.getFieldIndex(f).get()))); - } else { - partitionPathPositions.put(f, Collections.singletonList(-1)); - } - }); - // parse nested fields - getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains(".")) - .forEach(f -> partitionPathPositions.put(f, - RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false))); + getPartitionPathFields().stream().filter(f -> !f.isEmpty()) + .forEach(f -> partitionPathSchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, false))); } this.structType = structType; } } protected String getPartitionPathInternal(InternalRow row, StructType structType) { - buildFieldDataTypesMapIfNeeded(structType); + buildFieldSchemaInfoIfNeeded(structType); validatePartitionFieldsForInternalRow(); return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathPositions, partitionPathDataTypes); + hiveStylePartitioning, partitionPathSchemaInfo); } protected void validatePartitionFieldsForInternalRow() { - partitionPathPositions.entrySet().forEach(entry -> { - if (entry.getValue().size() > 1) { - throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns"); - } - }); - } - - void buildFieldDataTypesMapIfNeeded(StructType structType) { - buildFieldPositionMapIfNeeded(structType); - if (this.partitionPathDataTypes == null) { - this.partitionPathDataTypes = new HashMap<>(); - if (getPartitionPathFields() != null) { - // populating simple fields are good enough - getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains("."))) - .forEach(f -> { - if (structType.getFieldIndex(f).isDefined()) { - partitionPathDataTypes.put(f, - Collections.singletonList((structType.fields()[structType.fieldIndex(f)].dataType()))); - } else { - partitionPathDataTypes.put(f, Collections.singletonList(null)); - } - }); - } + if (!validatePartitionFields.getAndSet(true)) { + partitionPathSchemaInfo.values().forEach(entry -> { + if (entry.getKey().size() > 1) { + throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns"); + } + }); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java index 2e2167f9379f0..9ba3fb8760882 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java @@ -60,15 +60,15 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true); + buildFieldSchemaInfoIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true); } @Override public String getPartitionPath(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); + buildFieldSchemaInfoIfNeeded(row.schema()); return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathPositions); + hiveStylePartitioning, partitionPathSchemaInfo); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java index 391ea2c87c917..77eec748c7cb1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java @@ -60,8 +60,8 @@ public List getPartitionPathFields() { @Override public String getRecordKey(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true); + buildFieldSchemaInfoIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java index 032c750f03240..dc8b253b0f1be 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java @@ -61,6 +61,12 @@ public List getPartitionPathFields() { return nonpartitionedAvroKeyGenerator.getPartitionPathFields(); } + @Override + public String getRecordKey(Row row) { + buildFieldSchemaInfoIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); + } + @Override public String getPartitionPath(Row row) { return nonpartitionedAvroKeyGenerator.getEmptyPartition(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java index 6a28fbe9501a9..c0e10e6f9b775 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/RowKeyGeneratorHelper.java @@ -18,6 +18,7 @@ package org.apache.hudi.keygen; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieKeyException; import org.apache.spark.sql.Row; @@ -52,17 +53,18 @@ public class RowKeyGeneratorHelper { /** * Generates record key for the corresponding {@link Row}. - * @param row instance of {@link Row} of interest - * @param recordKeyFields record key fields as a list + * + * @param row instance of {@link Row} of interest + * @param recordKeyFields record key fields as a list * @param recordKeyPositions record key positions for the corresponding record keys in {@code recordKeyFields} - * @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise. + * @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise. * @return the record key thus generated */ - public static String getRecordKeyFromRow(Row row, List recordKeyFields, Map> recordKeyPositions, boolean prefixFieldName) { + public static String getRecordKeyFromRow(Row row, List recordKeyFields, Map, DataType>> recordKeyPositions, boolean prefixFieldName) { AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true); String toReturn = recordKeyFields.stream().map(field -> { String val = null; - List fieldPositions = recordKeyPositions.get(field); + List fieldPositions = recordKeyPositions.get(field).getKey(); if (fieldPositions.size() == 1) { // simple field Integer fieldPos = fieldPositions.get(0); if (row.isNullAt(fieldPos)) { @@ -76,7 +78,7 @@ public static String getRecordKeyFromRow(Row row, List recordKeyFields, } } } else { // nested fields - val = getNestedFieldVal(row, recordKeyPositions.get(field)).toString(); + val = getNestedFieldVal(row, recordKeyPositions.get(field).getKey()).toString(); if (!val.contains(NULL_RECORDKEY_PLACEHOLDER) && !val.contains(EMPTY_RECORDKEY_PLACEHOLDER)) { keyIsNullOrEmpty.set(false); } @@ -91,17 +93,18 @@ public static String getRecordKeyFromRow(Row row, List recordKeyFields, /** * Generates partition path for the corresponding {@link Row}. - * @param row instance of {@link Row} of interest - * @param partitionPathFields partition path fields as a list - * @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise + * + * @param row instance of {@link Row} of interest + * @param partitionPathFields partition path fields as a list + * @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise * @param partitionPathPositions partition path positions for the corresponding fields in {@code partitionPathFields} * @return the generated partition path for the row */ - public static String getPartitionPathFromRow(Row row, List partitionPathFields, boolean hiveStylePartitioning, Map> partitionPathPositions) { + public static String getPartitionPathFromRow(Row row, List partitionPathFields, boolean hiveStylePartitioning, Map, DataType>> partitionPathPositions) { return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> { String field = partitionPathFields.get(idx); String val = null; - List fieldPositions = partitionPathPositions.get(field); + List fieldPositions = partitionPathPositions.get(field).getKey(); if (fieldPositions.size() == 1) { // simple Integer fieldPos = fieldPositions.get(0); // for partition path, if field is not found, index will be set to -1 @@ -118,7 +121,7 @@ public static String getPartitionPathFromRow(Row row, List partitionPath val = field + "=" + val; } } else { // nested - Object data = getNestedFieldVal(row, partitionPathPositions.get(field)); + Object data = getNestedFieldVal(row, partitionPathPositions.get(field).getKey()); data = convertToTimestampIfInstant(data); if (data.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || data.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) { val = hiveStylePartitioning ? field + "=" + HUDI_DEFAULT_PARTITION_PATH : HUDI_DEFAULT_PARTITION_PATH; @@ -130,20 +133,20 @@ public static String getPartitionPathFromRow(Row row, List partitionPath }).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR)); } - public static String getPartitionPathFromInternalRow(InternalRow row, List partitionPathFields, boolean hiveStylePartitioning, - Map> partitionPathPositions, - Map> partitionPathDataTypes) { + public static String getPartitionPathFromInternalRow(InternalRow internalRow, List partitionPathFields, boolean hiveStylePartitioning, + Map, DataType>> partitionPathPositions) { return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> { String field = partitionPathFields.get(idx); String val = null; - List fieldPositions = partitionPathPositions.get(field); + List fieldPositions = partitionPathPositions.get(field).getKey(); + DataType dataType = partitionPathPositions.get(field).getValue(); if (fieldPositions.size() == 1) { // simple Integer fieldPos = fieldPositions.get(0); // for partition path, if field is not found, index will be set to -1 - if (fieldPos == -1 || row.isNullAt(fieldPos)) { + if (fieldPos == -1 || internalRow.isNullAt(fieldPos)) { val = HUDI_DEFAULT_PARTITION_PATH; } else { - Object value = row.get(fieldPos, partitionPathDataTypes.get(field).get(0)); + Object value = internalRow.get(fieldPos, dataType); if (value == null || value.toString().isEmpty()) { val = HUDI_DEFAULT_PARTITION_PATH; } else { @@ -180,22 +183,22 @@ public static Object getFieldValFromInternalRow(InternalRow internalRow, /** * Fetch the field value located at the positions requested for. - * + *

* The fetching logic recursively goes into the nested field based on the position list to get the field value. * For example, given the row [4357686,key1,2020-03-21,pi,[val1,10]] with the following schema, which has the fourth * field as a nested field, and positions list as [4,0], - * + *

* 0 = "StructField(timestamp,LongType,false)" * 1 = "StructField(_row_key,StringType,false)" * 2 = "StructField(ts_ms,StringType,false)" * 3 = "StructField(pii_col,StringType,false)" * 4 = "StructField(nested_col,StructType(StructField(prop1,StringType,false), StructField(prop2,LongType,false)),false)" - * + *

* the logic fetches the value from field nested_col.prop1. * If any level of the nested field is null, {@link KeyGenUtils#NULL_RECORDKEY_PLACEHOLDER} is returned. * If the field value is an empty String, {@link KeyGenUtils#EMPTY_RECORDKEY_PLACEHOLDER} is returned. * - * @param row instance of {@link Row} of interest + * @param row instance of {@link Row} of interest * @param positions tree style positions where the leaf node need to be fetched and returned * @return the field value as per the positions requested for. */ @@ -234,13 +237,14 @@ public static Object getNestedFieldVal(Row row, List positions) { * @param structType schema of interest * @param field field of interest for which the positions are requested for * @param isRecordKey {@code true} if the field requested for is a record key. {@code false} in case of a partition path. - * @return the positions of the field as per the struct type. + * @return the positions of the field as per the struct type and the leaf field's datatype. */ - public static List getNestedFieldIndices(StructType structType, String field, boolean isRecordKey) { + public static Pair, DataType> getFieldSchemaInfo(StructType structType, String field, boolean isRecordKey) { String[] slices = field.split("\\."); List positions = new ArrayList<>(); int index = 0; int totalCount = slices.length; + DataType leafFieldDataType = null; while (index < totalCount) { String slice = slices[index]; Option curIndexOpt = structType.getFieldIndex(slice); @@ -258,6 +262,9 @@ public static List getNestedFieldIndices(StructType structType, String } } structType = (StructType) nestedField.dataType(); + } else { + // leaf node. + leafFieldDataType = nestedField.dataType(); } } else { if (isRecordKey) { @@ -269,7 +276,7 @@ public static List getNestedFieldIndices(StructType structType, String } index++; } - return positions; + return Pair.of(positions, leafFieldDataType); } private static Object convertToTimestampIfInstant(Object data) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java index b84a8abdcc796..2f139a61eace8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java @@ -65,15 +65,15 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false); + buildFieldSchemaInfoIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); } @Override public String getPartitionPath(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); + buildFieldSchemaInfoIfNeeded(row.schema()); return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(), - hiveStylePartitioning, partitionPathPositions); + hiveStylePartitioning, partitionPathSchemaInfo); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java index e3a5a3310524b..004753f2461ae 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java @@ -29,8 +29,8 @@ import java.io.IOException; -import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH; import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER; +import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH; import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER; /** @@ -61,24 +61,24 @@ public String getPartitionPath(GenericRecord record) { @Override public String getRecordKey(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); - return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false); + buildFieldSchemaInfoIfNeeded(row.schema()); + return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false); } @Override public String getPartitionPath(Row row) { - buildFieldPositionMapIfNeeded(row.schema()); - Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0))); + buildFieldSchemaInfoIfNeeded(row.schema()); + Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey()); return getTimestampBasedPartitionPath(partitionPathFieldVal); } @Override public String getPartitionPath(InternalRow internalRow, StructType structType) { - buildFieldDataTypesMapIfNeeded(structType); + buildFieldSchemaInfoIfNeeded(structType); validatePartitionFieldsForInternalRow(); Object partitionPathFieldVal = RowKeyGeneratorHelper.getFieldValFromInternalRow(internalRow, - partitionPathPositions.get(getPartitionPathFields().get(0)).get(0), - partitionPathDataTypes.get(getPartitionPathFields().get(0)).get(0)); + partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey().get(0), + partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getValue()); return getTimestampBasedPartitionPath(partitionPathFieldVal); } 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 71efe89a055e1..20e3bd4c14ac3 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 @@ -63,7 +63,8 @@ public static HoodieSparkTable create(HoodieW HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) + .setProperties(config.getProps()).build(); return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java index e3d0e9b3c69d4..96ac794dcbc82 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java @@ -80,10 +80,11 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, Path so } catch (Exception e) { throw new HoodieException(e); } finally { - bootstrapHandle.close(); if (null != wrapper) { wrapper.shutdownNow(); + wrapper.awaitTermination(); } + bootstrapHandle.close(); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index d07ea771bc557..5f45629ba8023 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -68,9 +68,9 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException { void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception { BoundedInMemoryExecutor wrapper = null; + ParquetReader reader = + AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build(); try { - ParquetReader reader = - AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build(); wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { String recKey = keyGenerator.getKey(inp).getRecordKey(); @@ -84,10 +84,12 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, } catch (Exception e) { throw new HoodieException(e); } finally { - bootstrapHandle.close(); + reader.close(); if (null != wrapper) { wrapper.shutdownNow(); + wrapper.awaitTermination(); } + bootstrapHandle.close(); } } } 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 504da8a722810..4e488047d845e 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 @@ -334,7 +334,7 @@ private HoodieData runMetadataBootstrap(List getMetadataHandler(config, table, partitionFsPair.getRight().getRight()).runMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), keyGenerator)); 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 205da82ac145d..f8e4b31ff687e 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 @@ -113,7 +113,7 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, } private HoodieData> clusteringHandleUpdate(HoodieData> inputRecords, Set fileGroupsInPendingClustering) { - context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering"); + context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering: " + config.getTableName()); UpdateStrategy>> updateStrategy = (UpdateStrategy>>) ReflectionUtils .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); Pair>, Set> recordsAndPendingClusteringFileGroups = @@ -152,7 +152,7 @@ public HoodieWriteMetadata> execute(HoodieData> execute(HoodieData> inputRecordsWithClusteringUpdate = fileGroupsInPendingClustering.isEmpty() ? inputRecords : clusteringHandleUpdate(inputRecords, fileGroupsInPendingClustering); - context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data"); + context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data: " + config.getTableName()); HoodieData writeStatuses = mapPartitionsAsRDD(inputRecordsWithClusteringUpdate, partitioner); HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); updateIndexAndCommitIfNeeded(writeStatuses, result); @@ -280,7 +280,7 @@ protected void setCommitMetadata(HoodieWriteMetadata> re @Override protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { - context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect"); + context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect: " + config.getTableName()); commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collectAsList()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index c54c526253f0b..c2f5a43066d36 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -266,7 +266,7 @@ private Map> getSmallFilesForPartitions(List par } if (partitionPaths != null && partitionPaths.size() > 0) { - context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions"); + context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions: " + config.getTableName()); JavaRDD partitionPathRdds = jsc.parallelize(partitionPaths, partitionPaths.size()); partitionSmallFilesMap = partitionPathRdds.mapToPair((PairFunction>) partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath))).collectAsMap(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java new file mode 100644 index 0000000000000..e0bc22f70d231 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.functional; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.RealtimeFileStatus; +import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Test consistent hashing index + */ +@Tag("functional") +public class TestConsistentBucketIndex extends HoodieClientTestHarness { + + private final Random random = new Random(1); + private HoodieIndex index; + private HoodieWriteConfig config; + + private static Stream configParams() { + // preserveMetaField, partitioned + Object[][] data = new Object[][] { + {true, false}, + {false, false}, + {true, true}, + {false, true}, + }; + return Stream.of(data).map(Arguments::of); + } + + private void setUp(boolean populateMetaFields, boolean partitioned) throws Exception { + initPath(); + initSparkContexts(); + if (partitioned) { + initTestDataGenerator(); + } else { + initTestDataGenerator(new String[] {""}); + } + initFileSystem(); + Properties props = populateMetaFields ? new Properties() : getPropertiesForKeyGen(); + props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); + metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, props); + config = getConfigBuilder() + .withProperties(props) + .withIndexConfig(HoodieIndexConfig.newBuilder() + .fromProperties(props) + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withIndexKeyField("_row_key") + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING) + .build()) + .withAutoCommit(false) + .build(); + writeClient = getHoodieWriteClient(config); + index = writeClient.getIndex(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupResources(); + } + + /** + * Test bucket index tagging (always tag regardless of the write status) + * Test bucket index tagging consistency, two tagging result should be same + * + * @param populateMetaFields + * @param partitioned + * @throws Exception + */ + @ParameterizedTest + @MethodSource("configParams") + public void testTagLocation(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + String newCommitTime = "001"; + int totalRecords = 20 + random.nextInt(20); + List records = dataGen.generateInserts(newCommitTime, totalRecords); + JavaRDD writeRecords = jsc.parallelize(records, 2); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + + // The records should be tagged anyway, even though it is the first time doing tagging + List taggedRecord = tagLocation(index, writeRecords, hoodieTable).collect(); + Assertions.assertTrue(taggedRecord.stream().allMatch(r -> r.isCurrentLocationKnown())); + + // Tag again, the records should get the same location (hashing metadata has been persisted after the first tagging) + List taggedRecord2 = tagLocation(index, writeRecords, hoodieTable).collect(); + for (HoodieRecord ref : taggedRecord) { + for (HoodieRecord record : taggedRecord2) { + if (ref.getRecordKey().equals(record.getRecordKey())) { + Assertions.assertEquals(ref.getCurrentLocation(), record.getCurrentLocation()); + break; + } + } + } + } + + @ParameterizedTest + @MethodSource("configParams") + public void testWriteData(boolean populateMetaFields, boolean partitioned) throws Exception { + setUp(populateMetaFields, partitioned); + String newCommitTime = "001"; + int totalRecords = 20 + random.nextInt(20); + List records = dataGen.generateInserts(newCommitTime, totalRecords); + JavaRDD writeRecords = jsc.parallelize(records, 2); + + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Insert totalRecords records + writeClient.startCommitWithTime(newCommitTime); + List writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + boolean success = writeClient.commitStats(newCommitTime, writeStatues.stream() + .map(WriteStatus::getStat) + .collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + Assertions.assertTrue(success); + metaClient = HoodieTableMetaClient.reload(metaClient); + // The number of distinct fileId should be the same as total log file numbers + Assertions.assertEquals(writeStatues.stream().map(WriteStatus::getFileId).distinct().count(), + Arrays.stream(dataGen.getPartitionPaths()).mapToInt(p -> Objects.requireNonNull(listStatus(p, true)).length).sum()); + Assertions.assertEquals(totalRecords, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size()); + + // Upsert the same set of records, the number of records should be same + newCommitTime = "002"; + writeClient.startCommitWithTime(newCommitTime); + writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + success = writeClient.commitStats(newCommitTime, writeStatues.stream() + .map(WriteStatus::getStat) + .collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType()); + Assertions.assertTrue(success); + // The number of log file should double after this insertion + long numberOfLogFiles = Arrays.stream(dataGen.getPartitionPaths()) + .mapToInt(p -> { + return Arrays.stream(listStatus(p, true)).mapToInt(fs -> + fs instanceof RealtimeFileStatus ? ((RealtimeFileStatus) fs).getDeltaLogFiles().size() : 1).sum(); + }).sum(); + Assertions.assertEquals(writeStatues.stream().map(WriteStatus::getFileId).distinct().count() * 2, numberOfLogFiles); + // The record number should remain same because of deduplication + Assertions.assertEquals(totalRecords, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size()); + + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Upsert new set of records, and validate the total number of records + newCommitTime = "003"; + records = dataGen.generateInserts(newCommitTime, totalRecords); + writeRecords = jsc.parallelize(records, 2); + writeClient.startCommitWithTime(newCommitTime); + writeStatues = writeClient.upsert(writeRecords, newCommitTime).collect(); + org.apache.hudi.testutils.Assertions.assertNoWriteErrors(writeStatues); + success = writeClient.commitStats(newCommitTime, writeStatues.stream().map(WriteStatus::getStat).collect(Collectors.toList()), + Option.empty(), metaClient.getCommitActionType()); + Assertions.assertTrue(success); + Assertions.assertEquals(totalRecords * 2, readRecords(dataGen.getPartitionPaths(), populateMetaFields).size()); + } + + private List readRecords(String[] partitions, boolean populateMetaFields) { + return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, + Arrays.stream(partitions).map(p -> Paths.get(basePath, p).toString()).collect(Collectors.toList()), + basePath, new JobConf(hadoopConf), true, populateMetaFields); + } + + private FileStatus[] listStatus(String p, boolean realtime) { + JobConf jobConf = new JobConf(hadoopConf); + FileInputFormat.setInputPaths(jobConf, Paths.get(basePath, p).toString()); + FileInputFormat format = HoodieInputFormatUtils.getInputFormat(HoodieFileFormat.PARQUET, realtime, jobConf); + try { + if (realtime) { + return ((HoodieParquetRealtimeInputFormat) format).listStatus(jobConf); + } else { + return ((HoodieParquetInputFormat) format).listStatus(jobConf); + } + } catch (IOException e) { + e.printStackTrace(); + return null; + } + } + + private HoodieWriteConfig.Builder getConfigBuilder() { + return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2) + .withWriteStatusClass(MetadataMergeWriteStatus.class) + .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build()) + .forTable("test-trip-table") + .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build()); + } +} 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 024cf1ff50acc..8cbb74e6f5e03 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 @@ -131,6 +131,9 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean roll HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(indexType) .fromProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withIndexType(indexType); + if (indexType == IndexType.BUCKET) { + indexBuilder.withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE); + } config = getConfigBuilder() .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()) .withRollbackUsingMarkers(rollbackUsingMarkers) diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 91f9cbc96e6ed..a714d60d0033a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -28,6 +28,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; @@ -35,6 +36,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.Iterator; import java.util.List; import scala.Tuple2; @@ -105,6 +107,7 @@ protected Integer getResult() { } finally { if (executor != null) { executor.shutdownNow(); + executor.awaitTermination(); } } } @@ -152,7 +155,49 @@ protected Integer getResult() { } finally { if (executor != null) { executor.shutdownNow(); + executor.awaitTermination(); } } } + + @Test + public void testExecutorTermination() { + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + Iterator unboundedRecordIter = new Iterator() { + @Override + public boolean hasNext() { + return true; + } + + @Override + public GenericRecord next() { + return dataGen.generateGenericRecord(); + } + }; + + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { + @Override + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return 0; + } + }; + + BoundedInMemoryExecutor>, Integer> executor = + new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), unboundedRecordIter, + consumer, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA), + getPreExecuteRunnable()); + executor.shutdownNow(); + boolean terminatedGracefully = executor.awaitTermination(); + assertTrue(terminatedGracefully); + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java index 171403eb03847..b843546799479 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java @@ -26,7 +26,8 @@ import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.index.bloom.HoodieBloomIndex; import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex; -import org.apache.hudi.index.bucket.HoodieBucketIndex; +import org.apache.hudi.index.bucket.HoodieSimpleBucketIndex; +import org.apache.hudi.index.bucket.HoodieSparkConsistentBucketIndex; import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex; import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex; import org.apache.hudi.index.simple.HoodieSimpleIndex; @@ -88,8 +89,15 @@ public void testCreateIndex(IndexType indexType) { break; case BUCKET: config = clientConfigBuilder.withPath(basePath) - .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET).build()).build(); - assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBucketIndex); + .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE).build()).build(); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSimpleBucketIndex); + + config = clientConfigBuilder.withPath(basePath) + .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.CONSISTENT_HASHING).build()) + .build(); + assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieSparkConsistentBucketIndex); break; default: // no -op. just for checkstyle errors diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java similarity index 91% rename from hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java index 2b3765948bb63..c8b877cecad11 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bucket/TestHoodieSimpleBucketIndex.java @@ -52,10 +52,10 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -public class TestHoodieBucketIndex extends HoodieClientTestHarness { +public class TestHoodieSimpleBucketIndex extends HoodieClientTestHarness { - private static final Logger LOG = LogManager.getLogger(TestHoodieBucketIndex.class); - private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBucketIndex.class, "/exampleSchema.avsc", true); + private static final Logger LOG = LogManager.getLogger(TestHoodieSimpleBucketIndex.class); + private static final Schema SCHEMA = getSchemaFromResource(TestHoodieSimpleBucketIndex.class, "/exampleSchema.avsc", true); private static final int NUM_BUCKET = 8; @BeforeEach @@ -78,11 +78,15 @@ public void testBucketIndexValidityCheck() { props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "_row_key"); assertThrows(HoodieIndexException.class, () -> { HoodieIndexConfig.newBuilder().fromProperties(props) - .withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build(); + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE) + .withBucketNum("8").build(); }); props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "uuid"); HoodieIndexConfig.newBuilder().fromProperties(props) - .withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build(); + .withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE) + .withBucketNum("8").build(); } @Test @@ -110,7 +114,7 @@ public void testTagLocation() throws Exception { HoodieWriteConfig config = makeConfig(); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); - HoodieBucketIndex bucketIndex = new HoodieBucketIndex(config); + HoodieSimpleBucketIndex bucketIndex = new HoodieSimpleBucketIndex(config); HoodieData> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table); assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown())); @@ -133,6 +137,7 @@ private HoodieWriteConfig makeConfig() { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString()) .withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props) .withIndexType(HoodieIndex.IndexType.BUCKET) + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE) .withIndexKeyField("_row_key") .withBucketNum(String.valueOf(NUM_BUCKET)).build()).build(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java index b7f34ab2b24d8..66016305d7ad3 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java @@ -49,7 +49,7 @@ public void testGetFileWriter() throws IOException { SparkTaskContextSupplier supplier = new SparkTaskContextSupplier(); HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime, parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier); - assertTrue(parquetWriter instanceof HoodieParquetWriter); + assertTrue(parquetWriter instanceof HoodieAvroParquetWriter); // hfile format. final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile"); 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 8114daa30f763..30f7ad66543d1 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 @@ -148,7 +148,10 @@ private Properties makeIndexConfig(HoodieIndex.IndexType indexType) { props.putAll(indexConfig.build().getProps()); if (indexType.equals(HoodieIndex.IndexType.BUCKET)) { props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key"); - indexConfig.fromProperties(props).withIndexKeyField("_row_key").withBucketNum("1"); + indexConfig.fromProperties(props) + .withIndexKeyField("_row_key") + .withBucketNum("1") + .withBucketIndexEngineType(HoodieIndex.BucketIndexEngineType.SIMPLE); props.putAll(indexConfig.build().getProps()); props.putAll(HoodieLayoutConfig.newBuilder().fromProperties(props) .withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name()) @@ -419,7 +422,7 @@ public void testFileSizeUpsertRecords() throws Exception { List records = new ArrayList<>(); // Approx 1150 records are written for block size of 64KB - for (int i = 0; i < 2000; i++) { + for (int i = 0; i < 2050; i++) { String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}"; RawTripTestPayload rowChange = new RawTripTestPayload(recordStr); @@ -441,7 +444,8 @@ public void testFileSizeUpsertRecords() throws Exception { counts++; } } - assertEquals(5, counts, "If the number of records are more than 1150, then there should be a new file"); + // we check canWrite only once every 1000 records. and so 2 files with 1000 records and 3rd file with 50 records. + assertEquals(3, counts, "If the number of records are more than 1150, then there should be a new file"); } @Test diff --git a/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala b/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala index d4b89e1c9ec2e..cd55e381e2d7d 100644 --- a/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala +++ b/hudi-client/hudi-spark-client/src/test/scala/org/apache/hudi/keygen/TestRowGeneratorHelper.scala @@ -19,11 +19,9 @@ package org.apache.hudi.keygen import java.sql.Timestamp - import org.apache.spark.sql.Row - import org.apache.hudi.keygen.RowKeyGeneratorHelper._ - +import org.apache.spark.sql.types.{DataType, DataTypes} import org.junit.jupiter.api.{Assertions, Test} import scala.collection.JavaConverters._ @@ -36,7 +34,9 @@ class TestRowGeneratorHelper { /** single plain partition */ val row1 = Row.fromSeq(Seq(1, "z3", 10.0, "20220108")) val ptField1 = List("dt").asJava - val ptPos1 = Map("dt" -> List(new Integer(3)).asJava).asJava + val mapValue = org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.LongType) + val ptPos1 = Map("dt" -> mapValue).asJava + Assertions.assertEquals("20220108", getPartitionPathFromRow(row1, ptField1, false, ptPos1)) Assertions.assertEquals("dt=20220108", @@ -45,9 +45,9 @@ class TestRowGeneratorHelper { /** multiple plain partitions */ val row2 = Row.fromSeq(Seq(1, "z3", 10.0, "2022", "01", "08")) val ptField2 = List("year", "month", "day").asJava - val ptPos2 = Map("year" -> List(new Integer(3)).asJava, - "month" -> List(new Integer(4)).asJava, - "day" -> List(new Integer(5)).asJava + val ptPos2 = Map("year" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.StringType), + "month" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.StringType), + "day" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(5)).asJava, DataTypes.StringType) ).asJava Assertions.assertEquals("2022/01/08", getPartitionPathFromRow(row2, ptField2, false, ptPos2)) @@ -58,8 +58,8 @@ class TestRowGeneratorHelper { val timestamp = Timestamp.valueOf("2020-01-08 10:00:00") val instant = timestamp.toInstant val ptField3 = List("event", "event_time").asJava - val ptPos3 = Map("event" -> List(new Integer(3)).asJava, - "event_time" -> List(new Integer(4)).asJava + val ptPos3 = Map("event" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3)).asJava, DataTypes.StringType), + "event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.TimestampType) ).asJava // with timeStamp type @@ -79,7 +79,7 @@ class TestRowGeneratorHelper { /** mixed case with plain and nested partitions */ val nestedRow4 = Row.fromSeq(Seq(instant, "ad")) val ptField4 = List("event_time").asJava - val ptPos4 = Map("event_time" -> List(new Integer(3), new Integer(0)).asJava).asJava + val ptPos4 = Map("event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3), new Integer(0)).asJava, DataTypes.TimestampType)).asJava // with instant type val row4 = Row.fromSeq(Seq(1, "z3", 10.0, nestedRow4, "click")) Assertions.assertEquals("2020-01-08 10:00:00.0", @@ -90,8 +90,8 @@ class TestRowGeneratorHelper { val nestedRow5 = Row.fromSeq(Seq(timestamp, "ad")) val ptField5 = List("event", "event_time").asJava val ptPos5 = Map( - "event_time" -> List(new Integer(3), new Integer(0)).asJava, - "event" -> List(new Integer(4)).asJava + "event_time" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(3), new Integer(0)).asJava, DataTypes.TimestampType), + "event" -> org.apache.hudi.common.util.collection.Pair.of(List(new Integer(4)).asJava, DataTypes.StringType) ).asJava val row5 = Row.fromSeq(Seq(1, "z3", 10.0, nestedRow5, "click")) Assertions.assertEquals("click/2020-01-08 10:00:00.0", diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index f69d5683d1cfb..e2b586964ef4e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -400,7 +400,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor copyOldValueOrSetDefault(genericRecord, newRecord, f); } // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_METADATA_FIELD_POS, fileName); + newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); if (!GenericData.get().validate(newSchema, newRecord)) { throw new SchemaCompatibilityException( "Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema); @@ -412,7 +412,7 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) { GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>()); // do not preserve FILENAME_METADATA_FIELD - newRecord.put(HoodieRecord.FILENAME_METADATA_FIELD_POS, fileName); + newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName); return newRecord; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetastoreConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetastoreConfig.java new file mode 100644 index 0000000000000..36e2798a4d32a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetastoreConfig.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.config; + +import javax.annotation.concurrent.Immutable; +import java.util.Properties; + +/** + * Configurations used by the HUDI Metastore. + */ +@Immutable +@ConfigClassProperty(name = "Metastore Configs", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "Configurations used by the Hudi Metastore.") +public class HoodieMetastoreConfig extends HoodieConfig { + + public static final String METASTORE_PREFIX = "hoodie.metastore"; + + public static final ConfigProperty METASTORE_ENABLE = ConfigProperty + .key(METASTORE_PREFIX + ".enable") + .defaultValue(false) + .withDocumentation("Use metastore server to store hoodie table metadata"); + + public static final ConfigProperty METASTORE_URLS = ConfigProperty + .key(METASTORE_PREFIX + ".uris") + .defaultValue("thrift://localhost:9090") + .withDocumentation("Metastore server uris"); + + public static final ConfigProperty METASTORE_CONNECTION_RETRIES = ConfigProperty + .key(METASTORE_PREFIX + ".connect.retries") + .defaultValue(3) + .withDocumentation("Number of retries while opening a connection to metastore"); + + public static final ConfigProperty METASTORE_CONNECTION_RETRY_DELAY = ConfigProperty + .key(METASTORE_PREFIX + ".connect.retry.delay") + .defaultValue(1) + .withDocumentation("Number of seconds for the client to wait between consecutive connection attempts"); + + public static HoodieMetastoreConfig.Builder newBuilder() { + return new HoodieMetastoreConfig.Builder(); + } + + public boolean enableMetastore() { + return getBoolean(METASTORE_ENABLE); + } + + public String getMetastoreUris() { + return getStringOrDefault(METASTORE_URLS); + } + + public int getConnectionRetryLimit() { + return getIntOrDefault(METASTORE_CONNECTION_RETRIES); + } + + public int getConnectionRetryDelay() { + return getIntOrDefault(METASTORE_CONNECTION_RETRY_DELAY); + } + + public static class Builder { + private final HoodieMetastoreConfig config = new HoodieMetastoreConfig(); + + public Builder fromProperties(Properties props) { + this.config.getProps().putAll(props); + return this; + } + + public Builder setUris(String uris) { + config.setValue(METASTORE_URLS, uris); + return this; + } + + public HoodieMetastoreConfig build() { + config.setDefaults(HoodieMetastoreConfig.class.getName()); + return config; + } + } +} 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 09671ba2a3577..08015f61b2e04 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 @@ -18,6 +18,8 @@ package org.apache.hudi.common.config; +import org.apache.hudi.common.util.StringUtils; + import java.io.Serializable; import java.util.Arrays; import java.util.Enumeration; @@ -73,7 +75,7 @@ public List getStringList(String property, String delimiter, List !StringUtils.isNullOrEmpty(s)).collect(Collectors.toList()); } public int getInteger(String property) { 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 4e8d2b7eceaee..4b391ecbab752 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 @@ -77,6 +77,15 @@ public abstract class HoodieData implements Serializable { public abstract HoodieData mapPartitions( SerializableFunction, Iterator> func, boolean preservesPartitioning); + /** + * @param func serializable map function by taking a partition of objects + * and generating an iterator. + * @param output object type. + * @return {@link HoodieData} containing the result. Actual execution may be deferred. + */ + public abstract HoodieData mapPartitions( + SerializableFunction, Iterator> func); + /** * @param func serializable flatmap function. * @param output object type. 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 c23e712cf41ae..28ed2e282deb5 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 @@ -99,6 +99,11 @@ public HoodieData map(SerializableFunction func) { @Override public HoodieData mapPartitions(SerializableFunction, Iterator> func, boolean preservesPartitioning) { + return mapPartitions(func); + } + + @Override + public HoodieData mapPartitions(SerializableFunction, Iterator> func) { List result = new ArrayList<>(); throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add); return HoodieList.of(result); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 79badb48a5895..aa0cadf5b9354 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -348,6 +348,10 @@ public static String createNewFileIdPfx() { return UUID.randomUUID().toString(); } + public static String createNewFileId(String idPfx, int id) { + return String.format("%s-%d", idPfx, id); + } + /** * Get the file extension from the log file. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java new file mode 100644 index 0000000000000..262bb963223bb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ConsistentHashingNode.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.common.util.JsonUtils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** + * Used in consistent hashing index, representing nodes in the consistent hash ring. + * Record the end hash range value and its corresponding file group id. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ConsistentHashingNode implements Serializable { + + private final int value; + private final String fileIdPrefix; + + @JsonCreator + public ConsistentHashingNode(@JsonProperty("value") int value, @JsonProperty("fileIdPrefix") String fileIdPrefix) { + this.value = value; + this.fileIdPrefix = fileIdPrefix; + } + + public static String toJsonString(List nodes) throws IOException { + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(nodes); + } + + public static List fromJsonString(String json) throws Exception { + if (json == null || json.isEmpty()) { + return Collections.emptyList(); + } + + ConsistentHashingNode[] nodes = JsonUtils.getObjectMapper().readValue(json, ConsistentHashingNode[].class); + return Arrays.asList(nodes); + } + + public int getValue() { + return value; + } + + public String getFileIdPrefix() { + return fileIdPrefix; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("ConsistentHashingNode{"); + sb.append("value=").append(value); + sb.append(", fileIdPfx='").append(fileIdPrefix).append('\''); + sb.append('}'); + return sb.toString(); + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index 53ceb00409ac7..f5077dea859ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -18,17 +18,15 @@ package org.apache.hudi.common.model; -import com.fasterxml.jackson.annotation.JsonAutoDetect; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.JsonUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; + import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.PropertyAccessor; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -227,7 +225,7 @@ public String toJsonString() throws IOException { LOG.info("partition path is null for " + partitionToWriteStats.get(null)); partitionToWriteStats.remove(null); } - return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); } public static T fromJsonString(String jsonStr, Class clazz) throws Exception { @@ -235,7 +233,7 @@ public static T fromJsonString(String jsonStr, Class clazz) throws Except // For empty commit file (no data or somethings bad happen). return clazz.newInstance(); } - return getObjectMapper().readValue(jsonStr, clazz); + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); } // Here the functions are named "fetch" instead of "get", to get avoid of the json conversion. @@ -457,13 +455,6 @@ public static T fromBytes(byte[] bytes, Class clazz) throws IOException { } } - protected static ObjectMapper getObjectMapper() { - ObjectMapper mapper = new ObjectMapper(); - mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); - mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper; - } - @Override public String toString() { return "HoodieCommitMetadata{" + "partitionToWriteStats=" + partitionToWriteStats diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java new file mode 100644 index 0000000000000..46f115262745f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.JsonUtils; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * All the metadata that is used for consistent hashing bucket index + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieConsistentHashingMetadata implements Serializable { + + private static final Logger LOG = LogManager.getLogger(HoodieConsistentHashingMetadata.class); + /** + * Upper-bound of the hash value + */ + public static final int HASH_VALUE_MASK = Integer.MAX_VALUE; + public static final String HASHING_METADATA_FILE_SUFFIX = ".hashing_meta"; + + private final short version; + private final String partitionPath; + private final String instant; + private final int numBuckets; + private final int seqNo; + private final List nodes; + + @JsonCreator + public HoodieConsistentHashingMetadata(@JsonProperty("version") short version, @JsonProperty("partitionPath") String partitionPath, + @JsonProperty("instant") String instant, @JsonProperty("numBuckets") int numBuckets, + @JsonProperty("seqNo") int seqNo, @JsonProperty("nodes") List nodes) { + this.version = version; + this.partitionPath = partitionPath; + this.instant = instant; + this.numBuckets = numBuckets; + this.seqNo = seqNo; + this.nodes = nodes; + } + + /** + * Construct default metadata with all bucket's file group uuid initialized + */ + public HoodieConsistentHashingMetadata(String partitionPath, int numBuckets) { + this((short) 0, partitionPath, HoodieTimeline.INIT_INSTANT_TS, numBuckets, 0, constructDefaultHashingNodes(numBuckets)); + } + + private static List constructDefaultHashingNodes(int numBuckets) { + long step = ((long) HASH_VALUE_MASK + numBuckets - 1) / numBuckets; + return IntStream.range(1, numBuckets + 1) + .mapToObj(i -> new ConsistentHashingNode((int) Math.min(step * i, HASH_VALUE_MASK), FSUtils.createNewFileIdPfx())).collect(Collectors.toList()); + } + + public short getVersion() { + return version; + } + + public String getPartitionPath() { + return partitionPath; + } + + public String getInstant() { + return instant; + } + + public int getNumBuckets() { + return numBuckets; + } + + public int getSeqNo() { + return seqNo; + } + + public List getNodes() { + return nodes; + } + + public String getFilename() { + return instant + HASHING_METADATA_FILE_SUFFIX; + } + + public byte[] toBytes() throws IOException { + return toJsonString().getBytes(StandardCharsets.UTF_8); + } + + public static HoodieConsistentHashingMetadata fromBytes(byte[] bytes) throws IOException { + try { + return fromJsonString(new String(bytes, StandardCharsets.UTF_8), HoodieConsistentHashingMetadata.class); + } catch (Exception e) { + throw new IOException("unable to read hashing metadata", e); + } + } + + private String toJsonString() throws IOException { + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + } + + protected static T fromJsonString(String jsonStr, Class clazz) throws Exception { + if (jsonStr == null || jsonStr.isEmpty()) { + // For empty commit file (no data or something bad happen). + return clazz.newInstance(); + } + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); + } + + /** + * Get instant time from the hashing metadata filename + * Pattern of the filename: .HASHING_METADATA_FILE_SUFFIX + */ + public static String getTimestampFromFile(String filename) { + return filename.split("\\.")[0]; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java index 5b5a6432e633c..d4ad2cae1fe18 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java @@ -40,6 +40,9 @@ public class HoodieLogFile implements Serializable { public static final String DELTA_EXTENSION = ".log"; public static final Integer LOGFILE_BASE_VERSION = 1; + private static final Comparator LOG_FILE_COMPARATOR = new LogFileComparator(); + private static final Comparator LOG_FILE_COMPARATOR_REVERSED = new LogFileComparator().reversed(); + private transient FileStatus fileStatus; private final String pathStr; private long fileLen; @@ -129,11 +132,11 @@ public HoodieLogFile rollOver(FileSystem fs, String logWriteToken) throws IOExce } public static Comparator getLogFileComparator() { - return new LogFileComparator(); + return LOG_FILE_COMPARATOR; } public static Comparator getReverseLogFileComparator() { - return new LogFileComparator().reversed(); + return LOG_FILE_COMPARATOR_REVERSED; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index 0f21ae1bef185..e504b7b87dd9b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -42,8 +42,6 @@ public abstract class HoodieRecord implements Serializable { public static final String OPERATION_METADATA_FIELD = "_hoodie_operation"; public static final String HOODIE_IS_DELETED = "_hoodie_is_deleted"; - public static int FILENAME_METADATA_FIELD_POS = 4; - public static final List HOODIE_META_COLUMNS = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD, RECORD_KEY_METADATA_FIELD, PARTITION_PATH_METADATA_FIELD, FILENAME_METADATA_FIELD); @@ -59,6 +57,10 @@ public abstract class HoodieRecord implements Serializable { IntStream.range(0, HOODIE_META_COLUMNS.size()).mapToObj(idx -> Pair.of(HOODIE_META_COLUMNS.get(idx), idx)) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + public static int RECORD_KEY_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD); + public static int PARTITION_PATH_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD); + public static int FILENAME_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD); + /** * Identifies the record across the table. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java index 7cc9ee3a0c146..2dd6cda47d3db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java @@ -18,11 +18,9 @@ package org.apache.hudi.common.model; -import com.fasterxml.jackson.annotation.JsonAutoDetect; +import org.apache.hudi.common.util.JsonUtils; + import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import com.fasterxml.jackson.annotation.PropertyAccessor; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -80,7 +78,7 @@ public String toJsonString() throws IOException { LOG.info("partition path is null for " + partitionToReplaceFileIds.get(null)); partitionToReplaceFileIds.remove(null); } - return getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); } public static T fromJsonString(String jsonStr, Class clazz) throws Exception { @@ -88,7 +86,7 @@ public static T fromJsonString(String jsonStr, Class clazz) throws Except // For empty commit file (no data or somethings bad happen). return clazz.newInstance(); } - return getObjectMapper().readValue(jsonStr, clazz); + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); } @Override @@ -124,13 +122,6 @@ public static T fromBytes(byte[] bytes, Class clazz) throws IOException { } } - protected static ObjectMapper getObjectMapper() { - ObjectMapper mapper = new ObjectMapper(); - mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); - mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); - return mapper; - } - @Override public String toString() { return "HoodieReplaceMetadata{" + "partitionToWriteStats=" + partitionToWriteStats diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java index a354092675e4f..0a5240ed55d83 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.model; +import org.apache.hudi.common.util.JsonUtils; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -81,7 +83,7 @@ public String toJsonString() throws IOException { LOG.info("partition path is null for " + partitionToRollingStats.get(null)); partitionToRollingStats.remove(null); } - return HoodieCommitMetadata.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); } public HoodieRollingStatMetadata merge(HoodieRollingStatMetadata rollingStatMetadata) { 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 251a990d87c04..9945eb0650feb 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.table; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieMetastoreConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; @@ -38,6 +39,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; 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.exception.HoodieException; @@ -85,6 +87,7 @@ public class HoodieTableMetaClient implements Serializable { 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 HASHING_METADATA_FOLDER_NAME = ".bucket_index" + Path.SEPARATOR + "consistent_hashing_metadata"; public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR + ".partitions"; public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + Path.SEPARATOR @@ -97,21 +100,22 @@ public class HoodieTableMetaClient implements Serializable { // NOTE: Since those two parameters lay on the hot-path of a lot of computations, we // use tailored extension of the {@code Path} class allowing to avoid repetitive // computations secured by its immutability - private SerializablePath basePath; - private SerializablePath metaPath; + protected SerializablePath basePath; + protected SerializablePath metaPath; private transient HoodieWrapperFileSystem fs; private boolean loadActiveTimelineOnLoad; - private SerializableConfiguration hadoopConf; + protected SerializableConfiguration hadoopConf; private HoodieTableType tableType; private TimelineLayoutVersion timelineLayoutVersion; - private HoodieTableConfig tableConfig; - private HoodieActiveTimeline activeTimeline; + protected HoodieTableConfig tableConfig; + protected HoodieActiveTimeline activeTimeline; private HoodieArchivedTimeline archivedTimeline; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); + protected HoodieMetastoreConfig metastoreConfig; - private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, + protected HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig) { LOG.info("Loading HoodieTableMetaClient from " + basePath); @@ -211,6 +215,13 @@ public String getSchemaFolderName() { return new Path(metaPath.get(), SCHEMA_FOLDER_NAME).toString(); } + /** + * @return Hashing metadata base path + */ + public String getHashingMetadataPath() { + return new Path(metaPath.get(), HASHING_METADATA_FOLDER_NAME).toString(); + } + /** * @return Temp Folder path */ @@ -359,6 +370,13 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() { return archivedTimeline; } + public HoodieMetastoreConfig getMetastoreConfig() { + if (metastoreConfig == null) { + metastoreConfig = new HoodieMetastoreConfig(); + } + return metastoreConfig; + } + /** * Returns fresh new archived commits as a timeline from startTs (inclusive). * @@ -384,12 +402,14 @@ public void validateTableProperties(Properties properties) { throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back"); } - // Meta fields can be disabled only when {@code SimpleKeyGenerator} is used - if (!getTableConfig().populateMetaFields() - && !properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator") - .equals("org.apache.hudi.keygen.SimpleKeyGenerator")) { - throw new HoodieException("Only simple key generator is supported when meta fields are disabled. KeyGenerator used : " - + properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key())); + // meta fields can be disabled only with SimpleKeyGenerator, NonPartitioned and ComplexKeyGen. + if (!getTableConfig().populateMetaFields()) { + String keyGenClass = properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator"); + if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator") && !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator") + && !keyGenClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator")) { + throw new HoodieException("Only simple, non partitioned and complex key generator is supported when meta fields are disabled. KeyGenerator used : " + + properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key())); + } } } @@ -441,7 +461,8 @@ public static HoodieTableMetaClient initTableAndGetMetaClient(Configuration hado HoodieTableConfig.create(fs, metaPathDir, props); // We should not use fs.getConf as this might be different from the original configuration // used to create the fs in unit tests - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath) + .setProperties(props).build(); LOG.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType() + " from " + basePath); return metaClient; } @@ -610,6 +631,21 @@ public void initializeBootstrapDirsIfNotExists() throws IOException { initializeBootstrapDirsIfNotExists(getHadoopConf(), basePath.toString(), getFs()); } + private static HoodieTableMetaClient newMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, + ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, + String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig, Properties props) { + HoodieMetastoreConfig metastoreConfig = null == props + ? new HoodieMetastoreConfig.Builder().build() + : new HoodieMetastoreConfig.Builder().fromProperties(props).build(); + return metastoreConfig.enableMetastore() + ? (HoodieTableMetaClient) ReflectionUtils.loadClass("org.apache.hudi.common.table.HoodieTableMetastoreClient", + new Class[]{Configuration.class, ConsistencyGuardConfig.class, FileSystemRetryConfig.class, String.class, String.class, HoodieMetastoreConfig.class}, + conf, consistencyGuardConfig, fileSystemRetryConfig, + props.getProperty(HoodieTableConfig.DATABASE_NAME.key()), props.getProperty(HoodieTableConfig.NAME.key()), metastoreConfig) + : new HoodieTableMetaClient(conf, basePath, + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig); + } + public static Builder builder() { return new Builder(); } @@ -626,6 +662,7 @@ public static class Builder { private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); private Option layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION); + private Properties props; public Builder setConf(Configuration conf) { this.conf = conf; @@ -662,11 +699,16 @@ public Builder setLayoutVersion(Option layoutVersion) { return this; } + public Builder setProperties(Properties properties) { + this.props = properties; + return this; + } + public HoodieTableMetaClient build() { ValidationUtils.checkArgument(conf != null, "Configuration needs to be set to init HoodieTableMetaClient"); ValidationUtils.checkArgument(basePath != null, "basePath needs to be set to init HoodieTableMetaClient"); - return new HoodieTableMetaClient(conf, basePath, - loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig); + return newMetaClient(conf, basePath, + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig, props); } } 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 f178a23eeec7a..b76f71161d320 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 @@ -61,6 +61,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema; @@ -98,8 +99,8 @@ private MessageType getTableParquetSchemaFromDataFile() { // For COW table, the file has data written must be in parquet or orc format currently. if (instantAndCommitMetadata.isPresent()) { HoodieCommitMetadata commitMetadata = instantAndCommitMetadata.get().getRight(); - String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny().get(); - return readSchemaFromBaseFile(filePath); + Iterator filePaths = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().iterator(); + return fetchSchemaFromFiles(filePaths); } else { throw new IllegalArgumentException("Could not find any data file written for commit, " + "so could not get schema for table " + metaClient.getBasePath()); @@ -109,13 +110,8 @@ private MessageType getTableParquetSchemaFromDataFile() { // Determine the file format based on the file name, and then extract schema from it. if (instantAndCommitMetadata.isPresent()) { HoodieCommitMetadata commitMetadata = instantAndCommitMetadata.get().getRight(); - String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny().get(); - if (filePath.contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())) { - // this is a log file - return readSchemaFromLogFile(new Path(filePath)); - } else { - return readSchemaFromBaseFile(filePath); - } + Iterator filePaths = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().iterator(); + return fetchSchemaFromFiles(filePaths); } else { throw new IllegalArgumentException("Could not find any data file written for commit, " + "so could not get schema for table " + metaClient.getBasePath()); @@ -129,6 +125,20 @@ private MessageType getTableParquetSchemaFromDataFile() { } } + private MessageType fetchSchemaFromFiles(Iterator filePaths) throws IOException { + MessageType type = null; + while (filePaths.hasNext() && type == null) { + String filePath = filePaths.next(); + if (filePath.contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())) { + // this is a log file + type = readSchemaFromLogFile(new Path(filePath)); + } else { + type = readSchemaFromBaseFile(filePath); + } + } + return type; + } + private MessageType readSchemaFromBaseFile(String filePath) throws IOException { if (filePath.contains(HoodieFileFormat.PARQUET.getFileExtension())) { // this is a parquet file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index d912525fe9271..a62068e655e5d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -245,7 +245,7 @@ public void deleteInstantFileIfExists(HoodieInstant instant) { } } - private void deleteInstantFile(HoodieInstant instant) { + protected void deleteInstantFile(HoodieInstant instant) { LOG.info("Deleting instant " + instant); Path inFlightCommitFilePath = getInstantFileNamePath(instant.getFileName()); try { @@ -536,7 +536,7 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, transitionState(fromInstant, toInstant, data, false); } - private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, + protected void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, boolean allowRedundantTransitions) { ValidationUtils.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp())); try { @@ -566,7 +566,7 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, } } - private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { + protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); Path inFlightCommitFilePath = getInstantFileNamePath(inflight.getFileName()); Path commitFilePath = getInstantFileNamePath(completed.getFileName()); @@ -632,7 +632,7 @@ public void saveToCompactionRequested(HoodieInstant instant, Option cont } /** - * Saves content for inflight/requested REPLACE instant. + * Saves content for requested REPLACE instant. */ public void saveToPendingReplaceCommit(HoodieInstant instant, Option content) { ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); @@ -719,7 +719,7 @@ public void saveToPendingIndexAction(HoodieInstant instant, Option conte createFileInMetaPath(instant.getFileName(), content, false); } - private void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { + protected void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { Path fullPath = getInstantFileNamePath(filename); if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { FileIOUtils.createFileInPath(metaClient.getFs(), fullPath, content); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java index 4683fd6919ab4..35fda6c416ac7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java @@ -20,12 +20,14 @@ import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.HoodieMetastoreConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.function.SerializableSupplier; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Functions.Function2; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.metadata.HoodieMetadataFileSystemView; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -59,6 +61,8 @@ public class FileSystemViewManager { private static final Logger LOG = LogManager.getLogger(FileSystemViewManager.class); + private static final String HOODIE_METASTORE_FILE_SYSTEM_VIEW_CLASS = "org.apache.hudi.common.table.view.HoodieMetastoreFileSystemView"; + private final SerializableConfiguration conf; // The View Storage config used to store file-system views private final FileSystemViewStorageConfig viewStorageConfig; @@ -165,6 +169,11 @@ private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMeta return new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(), metadataSupplier.get()); } + if (metaClient.getMetastoreConfig().enableMetastore()) { + return (HoodieTableFileSystemView) ReflectionUtils.loadClass(HOODIE_METASTORE_FILE_SYSTEM_VIEW_CLASS, + new Class[] {HoodieTableMetaClient.class, HoodieTimeline.class, HoodieMetastoreConfig.class}, + metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metaClient.getMetastoreConfig()); + } return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled()); } @@ -184,6 +193,11 @@ public static HoodieTableFileSystemView createInMemoryFileSystemViewWithTimeline if (metadataConfig.enabled()) { return new HoodieMetadataFileSystemView(engineContext, metaClient, timeline, metadataConfig); } + if (metaClient.getMetastoreConfig().enableMetastore()) { + return (HoodieTableFileSystemView) ReflectionUtils.loadClass(HOODIE_METASTORE_FILE_SYSTEM_VIEW_CLASS, + new Class[] {HoodieTableMetaClient.class, HoodieTimeline.class, HoodieMetadataConfig.class}, + metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), metaClient.getMetastoreConfig()); + } return new HoodieTableFileSystemView(metaClient, timeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index 7330286734a08..c32e2cabb1012 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -124,7 +124,7 @@ Stream getLatestFileSlicesBeforeOrOn(String partitionPath, String max * @param maxInstantTime Max Instant Time * @return */ - public Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime); + Stream getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime); /** * Stream all the latest file slices, in the given range. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java b/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java index 0b82f091402a0..728ac717e4cd5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/Functions.java @@ -33,28 +33,28 @@ static Runnable noop() { /** * A function which has not any parameter. */ - public interface Function0 extends Serializable { + interface Function0 extends Serializable { R apply(); } /** * A function which contains only one parameter. */ - public interface Function1 extends Serializable { + interface Function1 extends Serializable { R apply(T1 val1); } /** * A function which contains two parameters. */ - public interface Function2 extends Serializable { + interface Function2 extends Serializable { R apply(T1 val1, T2 val2); } /** * A function which contains three parameters. */ - public interface Function3 extends Serializable { + interface Function3 extends Serializable { R apply(T1 val1, T2 val2, T3 val3); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java new file mode 100644 index 0000000000000..d820bde178e13 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/JsonUtils.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.util; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +public class JsonUtils { + + private static final ObjectMapper MAPPER = new ObjectMapper(); + static { + MAPPER.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + MAPPER.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + } + + public static ObjectMapper getObjectMapper() { + return MAPPER; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java index e156ccffdbb97..4915e454af215 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RateLimiter.java @@ -53,19 +53,22 @@ private RateLimiter(int permits, TimeUnit timePeriod) { } public boolean tryAcquire(int numPermits) { - if (numPermits > maxPermits) { - acquire(maxPermits); - return tryAcquire(numPermits - maxPermits); - } else { - return acquire(numPermits); + int remainingPermits = numPermits; + while (remainingPermits > 0) { + if (remainingPermits > maxPermits) { + acquire(maxPermits); + remainingPermits -= maxPermits; + } else { + return acquire(remainingPermits); + } } + return true; } public boolean acquire(int numOps) { try { - if (!semaphore.tryAcquire(numOps)) { + while (!semaphore.tryAcquire(numOps)) { Thread.sleep(WAIT_BEFORE_NEXT_ACQUIRE_PERMIT_IN_MS); - return acquire(numOps); } LOG.debug(String.format("acquire permits: %s, maxPremits: %s", numOps, maxPermits)); } catch (InterruptedException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HashID.java b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HashID.java index c56d76097866b..ccb29dfbb580d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HashID.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/hash/HashID.java @@ -106,6 +106,15 @@ public static byte[] hash(final byte[] messageBytes, final Size bits) { } } + public static int getXXHash32(final String message, int hashSeed) { + return getXXHash32(message.getBytes(StandardCharsets.UTF_8), hashSeed); + } + + public static int getXXHash32(final byte[] message, int hashSeed) { + XXHashFactory factory = XXHashFactory.fastestInstance(); + return factory.hash32().hash(message, 0, message.length, hashSeed); + } + private static byte[] getXXHash(final byte[] message, final Size bits) { XXHashFactory factory = XXHashFactory.fastestInstance(); switch (bits) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index d1e5e66083196..46ef5dc40caf8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -37,6 +37,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -48,7 +49,7 @@ public class BoundedInMemoryExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); - + private static final long TERMINATE_WAITING_TIME_SECS = 60L; // Executor service used for launching write thread. private final ExecutorService producerExecutorService; // Executor service used for launching read thread. @@ -168,6 +169,27 @@ public boolean isRemaining() { public void shutdownNow() { producerExecutorService.shutdownNow(); consumerExecutorService.shutdownNow(); + // close queue to force producer stop + queue.close(); + } + + public boolean awaitTermination() { + // if current thread has been interrupted before awaitTermination was called, we still give + // executor a chance to proceeding. So clear the interrupt flag and reset it if needed before return. + boolean interruptedBefore = Thread.interrupted(); + boolean producerTerminated = false; + boolean consumerTerminated = false; + try { + producerTerminated = producerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); + consumerTerminated = consumerExecutorService.awaitTermination(TERMINATE_WAITING_TIME_SECS, TimeUnit.SECONDS); + } catch (InterruptedException ie) { + // fail silently for any other interruption + } + // reset interrupt flag if needed + if (interruptedBefore) { + Thread.currentThread().interrupt(); + } + return producerTerminated && consumerTerminated; } public BoundedInMemoryQueue getQueue() { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieConsistentHashingMetadata.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieConsistentHashingMetadata.java new file mode 100644 index 0000000000000..8aa2e65561c59 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieConsistentHashingMetadata.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestHoodieConsistentHashingMetadata { + + @Test + public void testGetTimestamp() { + Assertions.assertTrue(HoodieConsistentHashingMetadata.getTimestampFromFile("0000.hashing_metadata").equals("0000")); + Assertions.assertTrue(HoodieConsistentHashingMetadata.getTimestampFromFile("1234.hashing_metadata").equals("1234")); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java index 311c131d432c6..dc64856d3c76c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java @@ -66,6 +66,10 @@ protected void initTestDataGenerator() { dataGen = new HoodieTestDataGenerator(); } + protected void initTestDataGenerator(String[] partitionPaths) { + dataGen = new HoodieTestDataGenerator(partitionPaths); + } + /** * Cleanups test data generator. * diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index cb4f5570743a6..e05d5f6f3e088 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -860,12 +860,14 @@ public boolean deleteExistingKeyIfPresent(HoodieKey key) { return false; } + public GenericRecord generateGenericRecord() { + return generateGenericRecord(genPseudoRandomUUID(rand).toString(), "0", + genPseudoRandomUUID(rand).toString(), genPseudoRandomUUID(rand).toString(), rand.nextLong()); + } + public List generateGenericRecords(int numRecords) { List list = new ArrayList<>(); - IntStream.range(0, numRecords).forEach(i -> { - list.add(generateGenericRecord(genPseudoRandomUUID(rand).toString(), "0", - genPseudoRandomUUID(rand).toString(), genPseudoRandomUUID(rand).toString(), rand.nextLong())); - }); + IntStream.range(0, numRecords).forEach(i -> list.add(generateGenericRecord())); return list; } diff --git a/hudi-flink-datasource/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 index c944f6a299144..729f0147b5940 100644 --- a/hudi-flink-datasource/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 @@ -709,25 +709,17 @@ private FlinkOptions() { // Prefix for Hoodie specific properties. private static final String PROPERTIES_PREFIX = "properties."; - /** - * Collects the config options that start with 'properties.' into a 'key'='value' list. - */ - public static Map getHoodieProperties(Map options) { - return getHoodiePropertiesWithPrefix(options, PROPERTIES_PREFIX); - } - /** * Collects the config options that start with specified prefix {@code prefix} into a 'key'='value' list. */ - public static Map getHoodiePropertiesWithPrefix(Map options, String prefix) { + public static Map getPropertiesWithPrefix(Map options, String prefix) { final Map hoodieProperties = new HashMap<>(); - - if (hasPropertyOptions(options)) { + if (hasPropertyOptions(options, prefix)) { options.keySet().stream() - .filter(key -> key.startsWith(PROPERTIES_PREFIX)) + .filter(key -> key.startsWith(prefix)) .forEach(key -> { final String value = options.get(key); - final String subKey = key.substring((prefix).length()); + final String subKey = key.substring(prefix.length()); hoodieProperties.put(subKey, value); }); } @@ -749,8 +741,8 @@ public static Configuration flatOptions(Configuration conf) { return fromMap(propsMap); } - private static boolean hasPropertyOptions(Map options) { - return options.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX)); + private static boolean hasPropertyOptions(Map options, String prefix) { + return options.keySet().stream().anyMatch(k -> k.startsWith(prefix)); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java new file mode 100644 index 0000000000000..72f20311504d0 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.configuration; + +import org.apache.flink.configuration.Configuration; +import org.apache.hudi.util.FlinkClientUtil; + +import java.util.Map; + +/** + * Utilities for fetching hadoop configurations. + */ +public class HadoopConfigurations { + private static final String HADOOP_PREFIX = "hadoop."; + private static final String PARQUET_PREFIX = "parquet."; + + /** + * Creates a merged hadoop configuration with given flink configuration and hadoop configuration. + */ + public static org.apache.hadoop.conf.Configuration getParquetConf( + org.apache.flink.configuration.Configuration options, + org.apache.hadoop.conf.Configuration hadoopConf) { + org.apache.hadoop.conf.Configuration copy = new org.apache.hadoop.conf.Configuration(hadoopConf); + Map parquetOptions = FlinkOptions.getPropertiesWithPrefix(options.toMap(), PARQUET_PREFIX); + parquetOptions.forEach((k, v) -> copy.set(PARQUET_PREFIX + k, v)); + return copy; + } + + /** + * Creates a new hadoop configuration that is initialized with the given flink configuration. + */ + public static org.apache.hadoop.conf.Configuration getHadoopConf(Configuration conf) { + org.apache.hadoop.conf.Configuration hadoopConf = FlinkClientUtil.getHadoopConf(); + Map options = FlinkOptions.getPropertiesWithPrefix(conf.toMap(), HADOOP_PREFIX); + options.forEach(hadoopConf::set); + return hadoopConf; + } +} diff --git a/hudi-flink-datasource/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 index 1443a68cf0fc2..a349314b7a111 100644 --- a/hudi-flink-datasource/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 @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.util.StreamerUtil; @@ -49,9 +50,10 @@ public static class Config { private Schema targetSchema; + @Deprecated public FilebasedSchemaProvider(TypedProperties props) { StreamerUtil.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP)); - FileSystem fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), StreamerUtil.getHadoopConf()); + FileSystem fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), HadoopConfigurations.getHadoopConf(new Configuration())); try { this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP)))); if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) { @@ -65,7 +67,7 @@ public FilebasedSchemaProvider(TypedProperties props) { public FilebasedSchemaProvider(Configuration conf) { final String sourceSchemaPath = conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH); - final FileSystem fs = FSUtils.getFs(sourceSchemaPath, StreamerUtil.getHadoopConf()); + final FileSystem fs = FSUtils.getFs(sourceSchemaPath, HadoopConfigurations.getHadoopConf(conf)); try { this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(sourceSchemaPath))); } catch (IOException ioe) { diff --git a/hudi-flink-datasource/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 index 023b1e696583a..39976e5ee2dc4 100644 --- a/hudi-flink-datasource/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 @@ -394,7 +394,7 @@ private void initInstant(String instant) { // starts a new instant startInstant(); // upgrade downgrade - this.writeClient.upgradeDowngrade(this.instant); + this.writeClient.upgradeDowngrade(this.instant, this.metaClient); }, "initialize instant %s", instant); } diff --git a/hudi-flink-datasource/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 index 1fc8d393be6a9..f0ef3bccb7fac 100644 --- a/hudi-flink-datasource/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 @@ -34,6 +34,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction; import org.apache.hudi.sink.meta.CkpMetadata; @@ -122,7 +123,7 @@ public void initializeState(StateInitializationContext context) throws Exception } } - this.hadoopConf = StreamerUtil.getHadoopConf(); + this.hadoopConf = HadoopConfigurations.getHadoopConf(this.conf); this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true); this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext()); this.ckpMetadata = CkpMetadata.getInstance(hoodieTable.getMetaClient().getFs(), this.writeConfig.getBasePath()); @@ -198,7 +199,7 @@ protected void loadRecords(String partitionPath) throws Exception { Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema(); List fileSlices = this.hoodieTable.getSliceView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp(), true) + .getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) .collect(toList()); for (FileSlice fileSlice : fileSlices) { diff --git a/hudi-flink-datasource/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 index 6c8dcef0f3925..06d9fcd851c22 100644 --- a/hudi-flink-datasource/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 @@ -113,7 +113,7 @@ public BulkInsertWriteFunction(Configuration config, RowType rowType) { public void open(Configuration parameters) throws IOException { this.taskID = getRuntimeContext().getIndexOfThisSubtask(); this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext()); - this.ckpMetadata = CkpMetadata.getInstance(config.getString(FlinkOptions.PATH)); + this.ckpMetadata = CkpMetadata.getInstance(config); this.initInstant = lastPendingInstant(); sendBootstrapEvent(); initWriterHelper(); diff --git a/hudi-flink-datasource/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 index b6fecff2042cc..3f84b2799ae56 100644 --- a/hudi-flink-datasource/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 @@ -18,6 +18,7 @@ package org.apache.hudi.sink.bulk; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.StringUtils; @@ -127,6 +128,10 @@ public static RowDataKeyGen instance(Configuration conf, RowType rowType) { keyGeneratorOpt); } + public HoodieKey getHoodieKey(RowData rowData) { + return new HoodieKey(getRecordKey(rowData), getPartitionPath(rowData)); + } + public String getRecordKey(RowData rowData) { if (this.simpleRecordKey) { return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0]); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java index 48d4f48989b0a..d5e718883b86c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.util.CompactionUtil; import org.apache.hudi.util.FlinkTables; @@ -88,7 +89,8 @@ public void notifyCheckpointComplete(long checkpointId) { // when the earliest inflight instant has timed out, assumes it has failed // already and just rolls it back. - CompactionUtil.rollbackEarliestCompaction(table, conf); + // comment out: do we really need the timeout rollback ? + // CompactionUtil.rollbackEarliestCompaction(table, conf); scheduleCompaction(table, checkpointId); } catch (Throwable throwable) { // make it fail-safe @@ -98,8 +100,7 @@ public void notifyCheckpointComplete(long checkpointId) { private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) throws IOException { // the first instant takes the highest priority. - HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - Option firstRequested = pendingCompactionTimeline + Option firstRequested = table.getActiveTimeline().filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).firstInstant(); if (!firstRequested.isPresent()) { // do nothing. @@ -107,13 +108,6 @@ private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) th return; } - Option firstInflight = pendingCompactionTimeline - .filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT).firstInstant(); - if (firstInflight.isPresent()) { - LOG.warn("Waiting for pending compaction instant : " + firstInflight + " to complete, skip scheduling new compaction plans"); - return; - } - String compactionInstantTime = firstRequested.get().getTimestamp(); // generate compaction plan @@ -134,6 +128,9 @@ private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) th List operations = compactionPlan.getOperations().stream() .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); LOG.info("Execute compaction plan for instant {} as {} file groups", compactionInstantTime, operations.size()); + WriteMarkersFactory + .get(table.getConfig().getMarkersType(), table, compactionInstantTime) + .deleteMarkerDir(table.getContext(), table.getConfig().getMarkersDeleteParallelism()); for (CompactionOperation operation : operations) { output.collect(new StreamRecord<>(new CompactionPlanEvent(compactionInstantTime, operation))); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java index ff1277d7b7e74..45a4e04bab285 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java @@ -18,11 +18,13 @@ package org.apache.hudi.sink.meta; +import org.apache.flink.configuration.Configuration; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.util.StreamerUtil; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -70,8 +72,8 @@ public class CkpMetadata implements Serializable { private List messages; private List instantCache; - private CkpMetadata(String basePath) { - this(FSUtils.getFs(basePath, StreamerUtil.getHadoopConf()), basePath); + private CkpMetadata(Configuration config) { + this(FSUtils.getFs(config.getString(FlinkOptions.PATH), HadoopConfigurations.getHadoopConf(config)), config.getString(FlinkOptions.PATH)); } private CkpMetadata(FileSystem fs, String basePath) { @@ -196,8 +198,8 @@ public boolean isAborted(String instant) { // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- - public static CkpMetadata getInstance(String basePath) { - return new CkpMetadata(basePath); + public static CkpMetadata getInstance(Configuration config) { + return new CkpMetadata(config); } public static CkpMetadata getInstance(FileSystem fs, String basePath) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java index c4b83bf51aace..89f89cf5c0a9f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssignFunction.java @@ -31,6 +31,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.sink.bootstrap.IndexRecord; import org.apache.hudi.sink.utils.PayloadCreation; import org.apache.hudi.table.action.commit.BucketInfo; @@ -116,7 +117,7 @@ public void open(Configuration parameters) throws Exception { super.open(parameters); HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true); HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( - new SerializableConfiguration(StreamerUtil.getHadoopConf()), + new SerializableConfiguration(HadoopConfigurations.getHadoopConf(this.conf)), new FlinkTaskContextSupplier(getRuntimeContext())); this.bucketAssigner = BucketAssigners.create( getRuntimeContext().getIndexOfThisSubtask(), diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java index b9b737ce22857..5fa3d1ab9a0a2 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.partitioner; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.index.bucket.BucketIdentifier; import org.apache.flink.api.common.functions.Partitioner; @@ -28,7 +29,7 @@ * * @param The type of obj to hash */ -public class BucketIndexPartitioner implements Partitioner { +public class BucketIndexPartitioner implements Partitioner { private final int bucketNum; private final String indexKeyFields; @@ -39,8 +40,9 @@ public BucketIndexPartitioner(int bucketNum, String indexKeyFields) { } @Override - public int partition(String key, int numPartitions) { + public int partition(HoodieKey key, int numPartitions) { int curBucket = BucketIdentifier.getBucketId(key, indexKeyFields, bucketNum); - return BucketIdentifier.mod(curBucket, numPartitions); + int globalHash = (key.getPartitionPath() + curBucket).hashCode() & Integer.MAX_VALUE; + return BucketIdentifier.mod(globalHash, numPartitions); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java index aad775a356423..d63696effba4a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java @@ -59,7 +59,7 @@ protected List smallFilesProfile(String partitionPath) { List allSmallFileSlices = new ArrayList<>(); // If we can index log files, we can add more inserts to log files for fileIds including those under // pending compaction. - List allFileSlices = fsView.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false) + List allFileSlices = fsView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) .collect(Collectors.toList()); for (FileSlice fileSlice : allFileSlices) { if (isSmallFile(fileSlice)) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index 536a0282fbcc4..bd837efc8737d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -22,11 +22,11 @@ import org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.table.format.FilePathUtils; -import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -60,7 +60,7 @@ public HiveSyncTool hiveSyncTool() { public static HiveSyncContext create(Configuration conf) { HiveSyncConfig syncConfig = buildSyncConfig(conf); - org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf(); + org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf); String path = conf.getString(FlinkOptions.PATH); FileSystem fs = FSUtils.getFs(path, hadoopConf); HiveConf hiveConf = new HiveConf(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java index 3b2ee39528a8b..91ac2beadc080 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/Pipelines.java @@ -18,6 +18,7 @@ package org.apache.hudi.sink.utils; +import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.OptionsResolver; @@ -96,13 +97,13 @@ public static DataStreamSink bulkInsert(Configuration conf, RowType rowT String indexKeys = conf.getString(FlinkOptions.INDEX_KEY_FIELD); int numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); - BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(numBuckets, indexKeys); + BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(numBuckets, indexKeys); RowDataKeyGen keyGen = RowDataKeyGen.instance(conf, rowType); RowType rowTypeWithFileId = BucketBulkInsertWriterHelper.rowTypeWithFileId(rowType); InternalTypeInfo typeInfo = InternalTypeInfo.of(rowTypeWithFileId); Map bucketIdToFileId = new HashMap<>(); - dataStream = dataStream.partitionCustom(partitioner, keyGen::getRecordKey) + dataStream = dataStream.partitionCustom(partitioner, keyGen::getHoodieKey) .map(record -> BucketBulkInsertWriterHelper.rowWithFileId(bucketIdToFileId, keyGen, record, indexKeys, numBuckets), typeInfo) .setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); // same parallelism as write task to avoid shuffle if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_INPUT)) { @@ -319,8 +320,8 @@ public static DataStream hoodieStreamWrite(Configuration conf, int defau WriteOperatorFactory operatorFactory = BucketStreamWriteOperator.getFactory(conf); int bucketNum = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD); - BucketIndexPartitioner partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields); - return dataStream.partitionCustom(partitioner, HoodieRecord::getRecordKey) + 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)); diff --git a/hudi-flink-datasource/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 index 07383ef7fea5f..d7125b414352d 100644 --- a/hudi-flink-datasource/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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.annotation.VisibleForTesting; @@ -54,7 +55,7 @@ public class FileIndex { private FileIndex(Path path, Configuration conf) { this.path = path; this.metadataConfig = metadataConfig(conf); - this.tableExists = StreamerUtil.tableExists(path.toString(), StreamerUtil.getHadoopConf()); + this.tableExists = StreamerUtil.tableExists(path.toString(), HadoopConfigurations.getHadoopConf(conf)); } public static FileIndex instance(Path path, Configuration conf) { diff --git a/hudi-flink-datasource/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 index 02e0e253cf577..94eeefcd36df3 100644 --- a/hudi-flink-datasource/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 @@ -226,7 +226,7 @@ public Result inputSplits( String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null); return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, endInstant, - metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, instantRange); + metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, instantRange, fileSlice.getFileId()); }).collect(Collectors.toList())) .flatMap(Collection::stream) .collect(Collectors.toList()); diff --git a/hudi-flink-datasource/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 index 8138e931e54e7..012bd093818a5 100644 --- a/hudi-flink-datasource/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 @@ -21,6 +21,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.StreamerUtil; @@ -157,7 +158,7 @@ public void initializeState(FunctionInitializationContext context) throws Except @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - this.hadoopConf = StreamerUtil.getHadoopConf(); + this.hadoopConf = HadoopConfigurations.getHadoopConf(conf); } @Override diff --git a/hudi-flink-datasource/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 index d00eb3e3ec700..1836857383554 100644 --- a/hudi-flink-datasource/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 @@ -20,13 +20,18 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.HoodieCommitMetadata; 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.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +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.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.HoodieROTablePathFilter; @@ -92,7 +97,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.hudi.table.format.FormatUtils.getParquetConf; +import static org.apache.hudi.configuration.HadoopConfigurations.getParquetConf; /** * Hoodie batch table source that always read the latest snapshot of the underneath table. @@ -155,7 +160,7 @@ public HoodieTableSource( : requiredPos; this.limit = limit == null ? NO_LIMIT_CONSTANT : limit; this.filters = filters == null ? Collections.emptyList() : filters; - this.hadoopConf = StreamerUtil.getHadoopConf(); + this.hadoopConf = HadoopConfigurations.getHadoopConf(conf); this.metaClient = StreamerUtil.metaClientForReader(conf, hadoopConf); this.maxCompactionMemoryInBytes = StreamerUtil.getMaxCompactionMemoryInBytes(conf); } @@ -181,6 +186,7 @@ public DataStream produceDataStream(StreamExecutionEnvironment execEnv) OneInputStreamOperatorFactory factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat); SingleOutputStreamOperator source = execEnv.addSource(monitoringFunction, getSourceOperatorName("split_monitor")) .setParallelism(1) + .keyBy(inputSplit -> inputSplit.getFileId()) .transform("split_reader", typeInfo, factory) .setParallelism(conf.getInteger(FlinkOptions.READ_TASKS)); return new DataStreamSource<>(source); @@ -316,7 +322,7 @@ private List buildFileIndex() { .map(logFile -> logFile.getPath().toString()) .collect(Collectors.toList())); return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit, - metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null); + metaClient.getBasePath(), maxCompactionMemoryInBytes, mergeType, null, fileSlice.getFileId()); }).collect(Collectors.toList())) .flatMap(Collection::stream) .collect(Collectors.toList()); @@ -379,8 +385,8 @@ private List buildFileIndex() { } private InputFormat getStreamInputFormat() { - // if table does not exist, use schema from the DDL - Schema tableAvroSchema = this.metaClient == null ? inferSchemaFromDdl() : getTableAvroSchema(); + // if table does not exist or table data does not exist, use schema from the DDL + Schema tableAvroSchema = (this.metaClient == null || !tableDataExists()) ? inferSchemaFromDdl() : getTableAvroSchema(); final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); final RowType rowType = (RowType) rowDataType.getLogicalType(); final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType(); @@ -397,6 +403,15 @@ private List buildFileIndex() { throw new HoodieException(errMsg); } + /** + * Returns whether the hoodie table data exists . + */ + private boolean tableDataExists() { + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + Option> instantAndCommitMetadata = activeTimeline.getLastCommitMetadataWithValidData(); + return instantAndCommitMetadata.isPresent(); + } + private MergeOnReadInputFormat mergeOnReadInputFormat( RowType rowType, RowType requiredRowType, diff --git a/hudi-flink-datasource/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 index 3317967006101..956d61cc3c2a4 100644 --- a/hudi-flink-datasource/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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.StreamerUtil; @@ -93,7 +94,7 @@ public class HoodieCatalog extends AbstractCatalog { public HoodieCatalog(String name, Configuration options) { super(name, options.get(DEFAULT_DATABASE)); this.catalogPathStr = options.get(CATALOG_PATH); - this.hadoopConf = StreamerUtil.getHadoopConf(); + this.hadoopConf = HadoopConfigurations.getHadoopConf(options); this.tableCommonOptions = CatalogOptions.tableCommonOptions(options); } diff --git a/hudi-flink-datasource/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 index fce9b75f764ea..478f94cb71f73 100644 --- a/hudi-flink-datasource/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 @@ -30,7 +30,6 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.config.HoodieWriteConfig; -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; @@ -49,7 +48,6 @@ import java.util.Iterator; import java.util.List; import java.util.Locale; -import java.util.Map; import java.util.function.Function; /** @@ -253,14 +251,4 @@ public static HoodieMergedLogRecordScanner logScanner( private static Boolean string2Boolean(String s) { return "true".equals(s.toLowerCase(Locale.ROOT)); } - - public static org.apache.hadoop.conf.Configuration getParquetConf( - org.apache.flink.configuration.Configuration options, - org.apache.hadoop.conf.Configuration hadoopConf) { - final String prefix = "parquet."; - org.apache.hadoop.conf.Configuration copy = new org.apache.hadoop.conf.Configuration(hadoopConf); - Map parquetOptions = FlinkOptions.getHoodiePropertiesWithPrefix(options.toMap(), prefix); - parquetOptions.forEach((k, v) -> copy.set(prefix + k, v)); - return copy; - } } diff --git a/hudi-flink-datasource/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 index 202b14404aa35..4f2de3648ed56 100644 --- a/hudi-flink-datasource/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 @@ -26,6 +26,7 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.keygen.KeyGenUtils; @@ -36,7 +37,6 @@ import org.apache.hudi.util.AvroToRowDataConverters; import org.apache.hudi.util.RowDataProjection; import org.apache.hudi.util.RowDataToAvroConverters; -import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.util.StringToRowDataConverter; import org.apache.avro.Schema; @@ -167,7 +167,7 @@ public static Builder builder() { public void open(MergeOnReadInputSplit split) throws IOException { this.currentReadCount = 0L; this.closed = false; - this.hadoopConf = StreamerUtil.getHadoopConf(); + this.hadoopConf = HadoopConfigurations.getHadoopConf(this.conf); if (!(split.getLogPaths().isPresent() && split.getLogPaths().get().size() > 0)) { if (split.getInstantRange() != null) { // base file only with commit time filtering @@ -306,7 +306,7 @@ private ParquetColumnarRowSplitReader getReader(String path, int[] requiredPos) return ParquetSplitReaderUtil.genPartColumnarRowReader( this.conf.getBoolean(FlinkOptions.UTC_TIMEZONE), true, - FormatUtils.getParquetConf(this.conf, hadoopConf), + HadoopConfigurations.getParquetConf(this.conf, hadoopConf), fieldNames.toArray(new String[0]), fieldTypes.toArray(new DataType[0]), partObjects, diff --git a/hudi-flink-datasource/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 index 156622c303519..cde646e41f035 100644 --- a/hudi-flink-datasource/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 @@ -43,6 +43,7 @@ public class MergeOnReadInputSplit implements InputSplit { private final long maxCompactionMemoryInBytes; private final String mergeType; private final Option instantRange; + private String fileId; // for streaming reader to record the consumed offset, // which is the start of next round reading. @@ -56,7 +57,8 @@ public MergeOnReadInputSplit( String tablePath, long maxCompactionMemoryInBytes, String mergeType, - @Nullable InstantRange instantRange) { + @Nullable InstantRange instantRange, + String fileId) { this.splitNum = splitNum; this.basePath = Option.ofNullable(basePath); this.logPaths = logPaths; @@ -65,6 +67,15 @@ public MergeOnReadInputSplit( this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes; this.mergeType = mergeType; this.instantRange = Option.ofNullable(instantRange); + this.fileId = fileId; + } + + public String getFileId() { + return fileId; + } + + public void setFileId(String fileId) { + this.fileId = fileId; } public Option getBasePath() { diff --git a/hudi-flink-datasource/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 index 6918a06b186b8..d440588b642e5 100644 --- a/hudi-flink-datasource/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 @@ -27,7 +27,7 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; -import static org.apache.hudi.util.StreamerUtil.getHadoopConf; +import static org.apache.hudi.configuration.HadoopConfigurations.getHadoopConf; import static org.apache.hudi.util.StreamerUtil.getHoodieClientConfig; /** @@ -44,7 +44,7 @@ private FlinkTables() { */ public static HoodieFlinkTable createTable(Configuration conf, RuntimeContext runtimeContext) { HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( - new SerializableConfiguration(getHadoopConf()), + new SerializableConfiguration(getHadoopConf(conf)), new FlinkTaskContextSupplier(runtimeContext)); HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true); return HoodieFlinkTable.create(writeConfig, context); diff --git a/hudi-flink-datasource/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 index dfbe0efd67c70..b977dfd7c5343 100644 --- a/hudi-flink-datasource/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 @@ -43,6 +43,7 @@ import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -101,7 +102,7 @@ public static TypedProperties getProps(FlinkStreamerConfig cfg) { return new TypedProperties(); } return readConfig( - getHadoopConf(), + HadoopConfigurations.getHadoopConf(cfg), new Path(cfg.propsFilePath), cfg.configs).getProps(); } @@ -140,11 +141,6 @@ public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Confi return conf; } - // Keep the redundant to avoid too many modifications. - public static org.apache.hadoop.conf.Configuration getHadoopConf() { - return FlinkClientUtil.getHadoopConf(); - } - /** * Mainly used for tests. */ @@ -215,7 +211,7 @@ public static HoodieWriteConfig getHoodieClientConfig( HoodieWriteConfig writeConfig = builder.build(); if (loadFsViewStorageConfig) { // do not use the builder to give a change for recovering the original fs view storage config - FileSystemViewStorageConfig viewStorageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH)); + FileSystemViewStorageConfig viewStorageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH), conf); writeConfig.setViewStorageConfig(viewStorageConfig); } return writeConfig; @@ -255,7 +251,7 @@ public static void checkRequiredProperties(TypedProperties props, List c */ public static HoodieTableMetaClient initTableIfNotExists(Configuration conf) throws IOException { final String basePath = conf.getString(FlinkOptions.PATH); - final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf(); + final org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf); if (!tableExists(basePath, hadoopConf)) { HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder() .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA)) @@ -348,18 +344,11 @@ public static HoodieTableMetaClient createMetaClient(String basePath, org.apache return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(hadoopConf).build(); } - /** - * Creates the meta client. - */ - public static HoodieTableMetaClient createMetaClient(String basePath) { - return createMetaClient(basePath, FlinkClientUtil.getHadoopConf()); - } - /** * Creates the meta client. */ public static HoodieTableMetaClient createMetaClient(Configuration conf) { - return createMetaClient(conf.getString(FlinkOptions.PATH)); + return createMetaClient(conf.getString(FlinkOptions.PATH), HadoopConfigurations.getHadoopConf(conf)); } /** @@ -382,7 +371,7 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf, Runti public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext, boolean loadFsViewStorageConfig) { HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( - new SerializableConfiguration(getHadoopConf()), + new SerializableConfiguration(HadoopConfigurations.getHadoopConf(conf)), new FlinkTaskContextSupplier(runtimeContext)); HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, loadFsViewStorageConfig); @@ -410,7 +399,7 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throw .withRemoteServerPort(viewStorageConfig.getRemoteViewServerPort()) .withRemoteTimelineClientTimeoutSecs(viewStorageConfig.getRemoteTimelineClientTimeoutSecs()) .build(); - ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), rebuilt); + ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), rebuilt, conf); return writeClient; } diff --git a/hudi-flink-datasource/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 index da55e27f0c03b..91662e47077c7 100644 --- a/hudi-flink-datasource/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 @@ -18,8 +18,10 @@ package org.apache.hudi.util; +import org.apache.flink.configuration.Configuration; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieIOException; import org.apache.hadoop.fs.FSDataInputStream; @@ -48,9 +50,10 @@ public class ViewStorageProperties { */ public static void createProperties( String basePath, - FileSystemViewStorageConfig config) throws IOException { + FileSystemViewStorageConfig config, + Configuration flinkConf) throws IOException { Path propertyPath = getPropertiesFilePath(basePath); - FileSystem fs = FSUtils.getFs(basePath, StreamerUtil.getHadoopConf()); + FileSystem fs = FSUtils.getFs(basePath, HadoopConfigurations.getHadoopConf(flinkConf)); fs.delete(propertyPath, false); try (FSDataOutputStream outputStream = fs.create(propertyPath)) { config.getProps().store(outputStream, @@ -61,10 +64,10 @@ public static void createProperties( /** * Read the {@link FileSystemViewStorageConfig} with given table base path. */ - public static FileSystemViewStorageConfig loadFromProperties(String basePath) { + public static FileSystemViewStorageConfig loadFromProperties(String basePath, Configuration conf) { Path propertyPath = getPropertiesFilePath(basePath); LOG.info("Loading filesystem view storage properties from " + propertyPath); - FileSystem fs = FSUtils.getFs(basePath, StreamerUtil.getHadoopConf()); + FileSystem fs = FSUtils.getFs(basePath, HadoopConfigurations.getHadoopConf(conf)); Properties props = new Properties(); try { try (FSDataInputStream inputStream = fs.open(propertyPath)) { diff --git a/hudi-flink-datasource/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 index 7a8aeff97b560..59a0580e56c5c 100644 --- a/hudi-flink-datasource/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 @@ -22,8 +22,11 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieWriteStat; 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.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.utils.MockCoordinatorExecutor; @@ -103,7 +106,7 @@ void testInstantState() { @Test public void testTableInitialized() throws IOException { - final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf(); + final org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(new Configuration()); String basePath = tempFile.getAbsolutePath(); try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) { assertTrue(fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))); @@ -201,7 +204,7 @@ void testSyncMetadataTable() throws Exception { assertNotEquals("", instant); final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath()); - HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath); + HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath, HadoopConfigurations.getHadoopConf(conf)); HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L)); assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); @@ -252,6 +255,49 @@ void testSyncMetadataTable() throws Exception { assertThat(completedTimeline.nthFromLastInstant(1).get().getAction(), is(HoodieTimeline.COMMIT_ACTION)); } + @Test + void testSyncMetadataTableWithReusedInstant() throws Exception { + // reset + reset(); + // override the default configuration + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + conf.setBoolean(FlinkOptions.METADATA_ENABLED, true); + OperatorCoordinator.Context context = new MockOperatorCoordinatorContext(new OperatorID(), 1); + coordinator = new StreamWriteOperatorCoordinator(conf, context); + coordinator.start(); + coordinator.setExecutor(new MockCoordinatorExecutor(context)); + + final WriteMetadataEvent event0 = WriteMetadataEvent.emptyBootstrap(0); + + coordinator.handleEventFromOperator(0, event0); + + String instant = coordinator.getInstant(); + assertNotEquals("", instant); + + final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath()); + HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath, HadoopConfigurations.getHadoopConf(conf)); + HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); + assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L)); + assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); + + // writes a normal commit + mockWriteWithMetadata(); + instant = coordinator.getInstant(); + // creates an inflight commit on the metadata timeline + metadataTableMetaClient.getActiveTimeline() + .createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieActiveTimeline.DELTA_COMMIT_ACTION, instant)); + metadataTableMetaClient.getActiveTimeline().transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instant); + metadataTableMetaClient.reloadActiveTimeline(); + + // write another commit with existing instant on the metadata timeline + instant = mockWriteWithMetadata(); + metadataTableMetaClient.reloadActiveTimeline(); + + completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); + assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(3L)); + assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/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 index 4771a7a3455b0..403d0272b4e18 100644 --- a/hudi-flink-datasource/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 @@ -352,6 +352,10 @@ public void testIndexStateBootstrap() throws Exception { // reset the config option conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, true); + validateIndexLoaded(); + } + + protected void validateIndexLoaded() throws Exception { preparePipeline(conf) .consume(TestData.DATA_SET_UPDATE_INSERT) .checkIndexLoaded( @@ -418,7 +422,7 @@ private TestHarness preparePipeline() throws Exception { return TestHarness.instance().preparePipeline(tempFile, conf); } - private TestHarness preparePipeline(Configuration conf) throws Exception { + protected TestHarness preparePipeline(Configuration conf) throws Exception { return TestHarness.instance().preparePipeline(tempFile, conf); } diff --git a/hudi-flink-datasource/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 index a35a0ac8d0b88..f2c0500f9555c 100644 --- a/hudi-flink-datasource/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 @@ -20,8 +20,10 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.utils.TestData; import org.apache.flink.configuration.Configuration; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; @@ -36,6 +38,27 @@ protected void setUp(Configuration conf) { conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); } + @Test + public void testIndexStateBootstrapWithCompactionScheduled() throws Exception { + // sets up the delta commits as 1 to generate a new compaction plan. + conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1); + // open the function and ingest data + preparePipeline(conf) + .consume(TestData.DATA_SET_INSERT) + .assertEmptyDataFiles() + .checkpoint(1) + .assertNextEvent() + .checkpointComplete(1) + .checkWrittenData(EXPECTED1, 4) + .end(); + + // reset config options + conf.removeConfig(FlinkOptions.COMPACTION_DELTA_COMMITS); + // sets up index bootstrap + conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, true); + validateIndexLoaded(); + } + @Override public void testInsertClustering() { // insert clustering is only valid for cow table. diff --git a/hudi-flink-datasource/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 index c4eecd7e4941b..a6fb493b9bdda 100644 --- a/hudi-flink-datasource/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 @@ -19,6 +19,7 @@ package org.apache.hudi.sink.meta; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; @@ -47,7 +48,7 @@ public class TestCkpMetadata { @BeforeEach public void beforeEach() throws Exception { String basePath = tempFile.getAbsolutePath(); - FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), StreamerUtil.getHadoopConf()); + FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(new Configuration())); Configuration conf = TestConfigurations.getDefaultConf(basePath); StreamerUtil.initTableIfNotExists(conf); diff --git a/hudi-flink-datasource/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 index 4f4b5499530cc..0748739064cf3 100644 --- a/hudi-flink-datasource/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 @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.sink.partitioner.profile.WriteProfile; import org.apache.hudi.table.action.commit.BucketInfo; import org.apache.hudi.table.action.commit.BucketType; @@ -71,7 +72,7 @@ public void before() throws IOException { writeConfig = StreamerUtil.getHoodieClientConfig(conf); context = new HoodieFlinkEngineContext( - new SerializableConfiguration(StreamerUtil.getHadoopConf()), + new SerializableConfiguration(HadoopConfigurations.getHadoopConf(conf)), new FlinkTaskContextSupplier(null)); StreamerUtil.initTableIfNotExists(conf); } diff --git a/hudi-flink-datasource/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 index a03f870296db7..ba60ff9469d73 100644 --- a/hudi-flink-datasource/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 @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.event.WriteMetadataEvent; @@ -345,7 +346,7 @@ public TestHarness checkWrittenData( } private void checkWrittenDataMor(File baseFile, Map expected, int partitions) throws Exception { - HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(basePath); + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(basePath, HadoopConfigurations.getHadoopConf(conf)); Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema(); String latestInstant = lastCompleteInstant(); FileSystem fs = FSUtils.getFs(basePath, new org.apache.hadoop.conf.Configuration()); diff --git a/hudi-flink-datasource/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 index db45a75977f5e..9f2aba77c1105 100644 --- a/hudi-flink-datasource/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 @@ -21,6 +21,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.format.mor.MergeOnReadInputFormat; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; @@ -239,7 +240,7 @@ private List generateSplits(StreamReadMonitoringFunction private OneInputStreamOperatorTestHarness createReader() throws Exception { final String basePath = tempFile.getAbsolutePath(); - final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf(); + final org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(new Configuration()); final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() .setConf(hadoopConf).setBasePath(basePath).build(); final List partitionKeys = Collections.singletonList("partition"); diff --git a/hudi-flink-datasource/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 index 088ddb260dd5f..0c423df6b7bdb 100644 --- a/hudi-flink-datasource/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 @@ -240,7 +240,15 @@ void testStreamWriteBatchReadOptimized() { List rows = CollectionUtil.iterableToList( () -> streamTableEnv.sqlQuery("select * from t1").execute().collect()); - assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT); + + // the test is flaky based on whether the first compaction is pending when + // scheduling the 2nd compaction. + // see details in CompactionPlanOperator#scheduleCompaction. + if (rows.size() < TestData.DATA_SET_SOURCE_INSERT.size()) { + assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_FIRST_COMMIT); + } else { + assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT); + } } @Test diff --git a/hudi-flink-datasource/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 index 3930e763fbaaa..8e23ef9d63bcb 100644 --- a/hudi-flink-datasource/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 @@ -42,6 +42,7 @@ import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -138,6 +139,13 @@ void beforeEach() { catalog.open(); } + @AfterEach + void afterEach() { + if (catalog != null) { + catalog.close(); + } + } + @Test public void testListDatabases() { List actual = catalog.listDatabases(); diff --git a/hudi-flink-datasource/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 index 6fbbab81fa4a6..8d2f3585cd942 100644 --- a/hudi-flink-datasource/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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.table.HoodieTableSource; import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat; import org.apache.hudi.table.format.mor.MergeOnReadInputFormat; @@ -400,7 +401,7 @@ void testReadIncrementally(HoodieTableType tableType) throws Exception { TestData.writeData(dataset, conf); } - HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath()); + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(conf)); List commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants() .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); diff --git a/hudi-flink-datasource/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 index c1e924056cfa2..c31c2bbadae25 100644 --- a/hudi-flink-datasource/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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -164,6 +165,18 @@ public class TestData { TimestampData.fromEpochMillis(8000), StringData.fromString("par4")) ); + // data set of test_source.data first commit. + public static List DATA_SET_SOURCE_INSERT_FIRST_COMMIT = Arrays.asList( + insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23, + TimestampData.fromEpochMillis(1000), StringData.fromString("par1")), + insertRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 33, + TimestampData.fromEpochMillis(2000), StringData.fromString("par1")), + insertRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53, + TimestampData.fromEpochMillis(3000), StringData.fromString("par2")), + insertRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31, + TimestampData.fromEpochMillis(4000), StringData.fromString("par2")) + ); + // data set of test_source.data latest commit. public static List DATA_SET_SOURCE_INSERT_LATEST_COMMIT = Arrays.asList( insertRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18, @@ -629,10 +642,11 @@ public static void checkWrittenDataMOR( File[] dataFiles = partitionDir.listFiles(file -> file.getName().contains(".log.") && !file.getName().startsWith("..")); assertNotNull(dataFiles); - HoodieMergedLogRecordScanner scanner = getScanner( - fs, baseFile.getPath(), Arrays.stream(dataFiles).map(File::getAbsolutePath) - .sorted(Comparator.naturalOrder()).collect(Collectors.toList()), - schema, latestInstant); + List logPaths = Arrays.stream(dataFiles) + .sorted((f1, f2) -> HoodieLogFile.getLogFileComparator() + .compare(new HoodieLogFile(f1.getPath()), new HoodieLogFile(f2.getPath()))) + .map(File::getAbsolutePath).collect(Collectors.toList()); + HoodieMergedLogRecordScanner scanner = getScanner(fs, baseFile.getPath(), logPaths, schema, latestInstant); List readBuffer = scanner.getRecords().values().stream() .map(hoodieRecord -> { try { diff --git a/hudi-flink-datasource/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 index 57297c50ee82b..43b59bdf9e8bc 100644 --- a/hudi-flink-datasource/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 @@ -106,7 +106,7 @@ void testInstantTimeDiff() { void testDumpRemoteViewStorageConfig() throws IOException { Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); StreamerUtil.createWriteClient(conf); - FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH)); + FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH), new Configuration()); assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST)); } } diff --git a/hudi-flink-datasource/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 index 466ccdfd01e72..c3aa9c25c61a2 100644 --- a/hudi-flink-datasource/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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.source.StreamReadMonitoringFunction; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.StreamerUtil; @@ -39,19 +40,19 @@ public class TestUtils { public static String getLastPendingInstant(String basePath) { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); return StreamerUtil.getLastPendingInstant(metaClient); } public static String getLastCompleteInstant(String basePath) { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); return StreamerUtil.getLastCompletedInstant(metaClient); } public static String getLastDeltaCompleteInstant(String basePath) { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); return metaClient.getCommitsTimeline().filterCompletedInstants() .filter(hoodieInstant -> hoodieInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) .lastInstant() @@ -61,7 +62,7 @@ public static String getLastDeltaCompleteInstant(String basePath) { public static String getFirstCompleteInstant(String basePath) { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant() .map(HoodieInstant::getTimestamp).orElse(null); } @@ -69,7 +70,7 @@ public static String getFirstCompleteInstant(String basePath) { @Nullable public static String getNthCompleteInstant(String basePath, int n, boolean isDelta) { final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() - .setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build(); + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); return metaClient.getActiveTimeline() .filterCompletedInstants() .filter(instant -> isDelta ? HoodieTimeline.DELTA_COMMIT_ACTION.equals(instant.getAction()) : HoodieTimeline.COMMIT_ACTION.equals(instant.getAction())) diff --git a/hudi-flink-datasource/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 index f80760bf1fd85..121a1c6785f30 100644 --- a/hudi-flink-datasource/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 @@ -18,6 +18,7 @@ package org.apache.hudi.utils; +import org.apache.flink.configuration.Configuration; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.util.ViewStorageProperties; @@ -45,11 +46,12 @@ void testReadWriteProperties() throws IOException { .withStorageType(FileSystemViewStorageType.SPILLABLE_DISK) .withRemoteServerHost("host1") .withRemoteServerPort(1234).build(); - ViewStorageProperties.createProperties(basePath, config); - ViewStorageProperties.createProperties(basePath, config); - ViewStorageProperties.createProperties(basePath, config); + Configuration flinkConfig = new Configuration(); + ViewStorageProperties.createProperties(basePath, config, flinkConfig); + ViewStorageProperties.createProperties(basePath, config, flinkConfig); + ViewStorageProperties.createProperties(basePath, config, flinkConfig); - FileSystemViewStorageConfig readConfig = ViewStorageProperties.loadFromProperties(basePath); + FileSystemViewStorageConfig readConfig = ViewStorageProperties.loadFromProperties(basePath, new Configuration()); assertThat(readConfig.getStorageType(), is(FileSystemViewStorageType.SPILLABLE_DISK)); assertThat(readConfig.getRemoteViewServerHost(), is("host1")); assertThat(readConfig.getRemoteViewServerPort(), is(1234)); diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index 6c1bad138cc18..5d26d03a20a89 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -522,6 +522,78 @@ Spark submit with the flag: --saferSchemaEvolution ``` +### Multi-writer tests +Integ test framework also supports multi-writer tests. + +#### Multi-writer tests with deltastreamer and a spark data source writer. + +Sample spark-submit command to test one delta streamer and a spark data source writer. +```shell +./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.0 \ +--conf spark.task.cpus=3 --conf spark.executor.cores=3 \ +--conf spark.task.maxFailures=100 --conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true --conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 --conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false --conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s --conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 \ +--conf spark.sql.catalogImplementation=hive \ +--class org.apache.hudi.integ.testsuite.HoodieMultiWriterTestSuiteJob \ +/packaging/hudi-integ-test-bundle/target/hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /tmp/hudi/output \ +--input-base-paths "/tmp/hudi/input1,/tmp/hudi/input2" \ +--target-table table1 \ +--props-paths "file:/docker/demo/config/test-suite/multi-writer-local-1.properties,file:/hudi/docker/demo/config/test-suite/multi-writer-local-2.properties" \ +--schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-paths "file:/docker/demo/config/test-suite/multi-writer-1-ds.yaml,file:/docker/demo/config/test-suite/multi-writer-2-sds.yaml" \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 \ +--input-base-path "dummyValue" \ +--workload-yaml-path "dummyValue" \ +--props "dummyValue" \ +--use-hudi-data-to-generate-updates +``` + +#### Multi-writer tests with 4 concurrent spark data source writer. + +```shell +./bin/spark-submit --packages org.apache.spark:spark-avro_2.11:2.4.0 \ +--conf spark.task.cpus=3 --conf spark.executor.cores=3 \ +--conf spark.task.maxFailures=100 --conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true --conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 --conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false --conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s --conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 --conf spark.sql.catalogImplementation=hive \ +--class org.apache.hudi.integ.testsuite.HoodieMultiWriterTestSuiteJob \ +/hudi-integ-test-bundle-0.12.0-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /tmp/hudi/output \ +--input-base-paths "/tmp/hudi/input1,/tmp/hudi/input2,/tmp/hudi/input3,/tmp/hudi/input4" \ +--target-table table1 \ +--props-paths "file:/multi-writer-local-1.properties,file:/multi-writer-local-2.properties,file:/multi-writer-local-3.properties,file:/multi-writer-local-4.properties" +--schemaprovider-class org.apache.hudi.integ.testsuite.schema.TestSuiteFileBasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-paths "file:/multi-writer-1-sds.yaml,file:/multi-writer-2-sds.yaml,file:/multi-writer-3-sds.yaml,file:/multi-writer-4-sds.yaml" \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 \ +--input-base-path "dummyValue" \ +--workload-yaml-path "dummyValue" \ +--props "dummyValue" \ +--use-hudi-data-to-generate-updates +``` + + ## Automated tests for N no of yamls in Local Docker environment Hudi provides a script to assist you in testing N no of yamls automatically. Checkout the script under diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieMultiWriterTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieMultiWriterTestSuiteJob.java index 6cff499825566..87d2f587597a0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieMultiWriterTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieMultiWriterTestSuiteJob.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -116,6 +117,7 @@ public static void main(String[] args) throws Exception { } ExecutorService executor = Executors.newFixedThreadPool(inputPaths.length); + Random random = new Random(); List testSuiteConfigList = new ArrayList<>(); int jobIndex = 0; @@ -131,11 +133,20 @@ public static void main(String[] args) throws Exception { AtomicBoolean jobFailed = new AtomicBoolean(false); AtomicInteger counter = new AtomicInteger(0); + List waitTimes = new ArrayList<>(); + for (int i = 0;i < jobIndex ;i++) { + if (i == 0) { + waitTimes.add(0L); + } else { + // every job after 1st, will start after 1 min + some delta. + waitTimes.add(60000L + random.nextInt(10000)); + } + } List> completableFutureList = new ArrayList<>(); testSuiteConfigList.forEach(hoodieTestSuiteConfig -> { try { // start each job at 20 seconds interval so that metaClient instantiation does not overstep - Thread.sleep(counter.get() * 20000); + Thread.sleep(waitTimes.get(counter.get())); LOG.info("Starting job " + hoodieTestSuiteConfig.toString()); } catch (InterruptedException e) { e.printStackTrace(); 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 581cce954a53c..1578e86be47b6 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 @@ -101,6 +101,8 @@ public static class Config { private static String ENABLE_METADATA_VALIDATE = "enable_metadata_validate"; private static String VALIDATE_FULL_DATA = "validate_full_data"; private static String DELETE_INPUT_DATA_EXCEPT_LATEST = "delete_input_data_except_latest"; + private static String PARTITIONS_TO_DELETE = "partitions_to_delete"; + private static String INPUT_PARTITIONS_TO_SKIP_VALIDATE = "input_partitions_to_skip_validate"; // Spark SQL Create Table private static String TABLE_TYPE = "table_type"; @@ -203,6 +205,10 @@ public boolean isDisableIngest() { return Boolean.valueOf(configsMap.getOrDefault(DISABLE_INGEST, false).toString()); } + public String getPartitionsToDelete() { + return configsMap.getOrDefault(PARTITIONS_TO_DELETE, "").toString(); + } + public boolean getReinitContext() { return Boolean.valueOf(configsMap.getOrDefault(REINIT_CONTEXT, false).toString()); } @@ -223,6 +229,10 @@ public int validateOnceEveryIteration() { return Integer.valueOf(configsMap.getOrDefault(VALIDATE_ONCE_EVERY_ITR, 1).toString()); } + public String inputPartitonsToSkipWithValidate() { + return configsMap.getOrDefault(INPUT_PARTITIONS_TO_SKIP_VALIDATE, "").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 b5c661cb085f6..a0ebdc5754716 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 @@ -163,8 +163,13 @@ private Dataset getInputDf(ExecutionContext context, SparkSession session, // todo: fix hard coded fields from configs. // read input and resolve insert, updates, etc. Dataset inputDf = session.read().format("avro").load(inputPath); + Dataset trimmedDf = inputDf; + if (!config.inputPartitonsToSkipWithValidate().isEmpty()) { + trimmedDf = inputDf.filter("instr("+partitionPathField+", \'"+ config.inputPartitonsToSkipWithValidate() +"\') != 1"); + } + ExpressionEncoder encoder = getEncoder(inputDf.schema()); - return inputDf.groupByKey( + return trimmedDf.groupByKey( (MapFunction) value -> (partitionPathField.isEmpty() ? value.getAs(recordKeyField) : (value.getAs(partitionPathField) + "+" + value.getAs(recordKeyField))), Encoders.STRING()) .reduceGroups((ReduceFunction) (v1, v2) -> { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java index 0183f52c2a17a..ab80df0d6a1db 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java @@ -117,9 +117,6 @@ private void execute(ExecutorService service, WorkflowDag workflowDag) throws Ex if (curRound < workflowDag.getRounds()) { new DelayNode(workflowDag.getIntermittentDelayMins()).execute(executionContext, curRound); } - - // After each level, report and flush the metrics - Metrics.flush(); } while (curRound++ < workflowDag.getRounds()); log.info("Finished workloads"); } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index e7bc7b00a82a4..c30be2a2a5d2c 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -123,7 +123,7 @@ public JavaRDD generateInserts(Config operation) { int startPartition = operation.getStartPartition(); // Each spark partition below will generate records for a single partition given by the integer index. - List partitionIndexes = IntStream.rangeClosed(0 + startPartition, numPartitions + startPartition) + List partitionIndexes = IntStream.rangeClosed(0 + startPartition, numPartitions + startPartition - 1) .boxed().collect(Collectors.toList()); JavaRDD inputBatch = jsc.parallelize(partitionIndexes, numPartitions) diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java index 59f02de0ac1a6..a936a81665116 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java @@ -338,7 +338,7 @@ public boolean validate(GenericRecord record) { */ @VisibleForTesting public GenericRecord updateTimestamp(GenericRecord record, String fieldName) { - long delta = TimeUnit.SECONDS.convert((++partitionIndex % numDatePartitions) + startPartition, TimeUnit.DAYS); + long delta = TimeUnit.SECONDS.convert((partitionIndex++ % numDatePartitions) + startPartition, TimeUnit.DAYS); record.put(fieldName, delta); return record; } diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala index ac254bea8dad0..b426f87071127 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala @@ -19,49 +19,18 @@ package org.apache.hudi.integ.testsuite.dag.nodes import org.apache.hudi.client.WriteStatus -import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config -import org.apache.hudi.integ.testsuite.dag.ExecutionContext -import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions} +import org.apache.hudi.DataSourceWriteOptions import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SaveMode - -import scala.collection.JavaConverters._ /** * Spark datasource based bulk insert node * * @param dagNodeConfig DAG node configurations. */ -class SparkBulkInsertNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { - - config = dagNodeConfig +class SparkBulkInsertNode(dagNodeConfig: Config) extends SparkInsertNode(dagNodeConfig) { - /** - * Execute the {@link DagNode}. - * - * @param context The context needed for an execution of a node. - * @param curItrCount iteration count for executing the node. - * @throws Exception Thrown if the execution failed. - */ - override def execute(context: ExecutionContext, curItrCount: Int): Unit = { - if (!config.isDisableGenerate) { - context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).getValue().count() - } - val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch, - context.getWriterContext.getHoodieTestSuiteWriter.getSchema, - context.getWriterContext.getSparkSession) - val saveMode = if(curItrCount == 0) SaveMode.Overwrite else SaveMode.Append - inputDF.write.format("hudi") - .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) - .option(DataSourceWriteOptions.TABLE_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName) - .option(DataSourceWriteOptions.TABLE_TYPE.key(), context.getHoodieTestSuiteWriter.getCfg.tableType) - .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.ENABLE_ROW_WRITER.key(), String.valueOf(config.enableRowWriting())) - .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key(), "deltastreamer.checkpoint.key") - .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse("")) - .option(HoodieWriteConfig.TBL_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName) - .mode(saveMode) - .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + override def getOperation(): String = { + DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL } } diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala new file mode 100644 index 0000000000000..9354deea28bb0 --- /dev/null +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes + + +import org.apache.avro.Schema +import org.apache.hudi.client.WriteStatus +import org.apache.hudi.common.util.collection.Pair +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config +import org.apache.hudi.integ.testsuite.dag.ExecutionContext +import org.apache.hudi.integ.testsuite.schema.SchemaUtils +import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats +import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkUtils} +import org.apache.log4j.LogManager +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SaveMode + +import scala.collection.JavaConverters._ + +/** + * Spark datasource based insert node + * + * @param dagNodeConfig DAG node configurations. + */ +class SparkDeletePartitionNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { + + private val log = LogManager.getLogger(getClass) + config = dagNodeConfig + + /** + * Execute the {@link DagNode}. + * + * @param context The context needed for an execution of a node. + * @param curItrCount iteration count for executing the node. + * @throws Exception Thrown if the execution failed. + */ + override def execute(context: ExecutionContext, curItrCount: Int): Unit = { + println("Generating input data for node {}", this.getName) + + context.getWriterContext.getSparkSession.emptyDataFrame.write.format("hudi") + .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), SchemaUtils.SOURCE_ORDERING_FIELD) + .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL) + .option(HoodieWriteConfig.TBL_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) + .option(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key, config.getPartitionsToDelete) + .mode(SaveMode.Append) + .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath) + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java index b3acf444adb88..0ed7b23c7471a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java @@ -22,7 +22,11 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.hive.NonPartitionedExtractor; import org.apache.hudi.keygen.BuiltinKeyGenerator; +import org.apache.hudi.keygen.ComplexKeyGenerator; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.log4j.LogManager; @@ -57,18 +61,18 @@ public class HoodieDatasetBulkInsertHelper { /** * Prepares input hoodie spark dataset for bulk insert. It does the following steps. - * 1. Uses KeyGenerator to generate hoodie record keys and partition path. - * 2. Add hoodie columns to input spark dataset. - * 3. Reorders input dataset columns so that hoodie columns appear in the beginning. - * 4. Sorts input dataset by hoodie partition path and record key + * 1. Uses KeyGenerator to generate hoodie record keys and partition path. + * 2. Add hoodie columns to input spark dataset. + * 3. Reorders input dataset columns so that hoodie columns appear in the beginning. + * 4. Sorts input dataset by hoodie partition path and record key * * @param sqlContext SQL Context - * @param config Hoodie Write Config - * @param rows Spark Input dataset + * @param config Hoodie Write Config + * @param rows Spark Input dataset * @return hoodie dataset which is ready for bulk insert. */ public static Dataset prepareHoodieDatasetForBulkInsert(SQLContext sqlContext, - HoodieWriteConfig config, Dataset rows, String structName, String recordNamespace, + HoodieWriteConfig config, Dataset rows, String structName, String recordNamespace, BulkInsertPartitioner> bulkInsertPartitionerRows, boolean isGlobalIndex, boolean dropPartitionColumns) { List originalFields = @@ -77,27 +81,46 @@ public static Dataset prepareHoodieDatasetForBulkInsert(SQLContext sqlConte TypedProperties properties = new TypedProperties(); properties.putAll(config.getProps()); String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key()); + String recordKeyFields = properties.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()); + String partitionPathFields = properties.containsKey(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()) + ? properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()) : ""; BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties); - String tableName = properties.getString(HoodieWriteConfig.TBL_NAME.key()); - String recordKeyUdfFn = RECORD_KEY_UDF_FN + tableName; - String partitionPathUdfFn = PARTITION_PATH_UDF_FN + tableName; - sqlContext.udf().register(recordKeyUdfFn, (UDF1) keyGenerator::getRecordKey, DataTypes.StringType); - sqlContext.udf().register(partitionPathUdfFn, (UDF1) keyGenerator::getPartitionPath, DataTypes.StringType); - - final Dataset rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, - callUDF(recordKeyUdfFn, org.apache.spark.sql.functions.struct( - JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq()))); - - final Dataset rowDatasetWithRecordKeysAndPartitionPath = - rowDatasetWithRecordKeys.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, - callUDF(partitionPathUdfFn, - org.apache.spark.sql.functions.struct( - JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq()))); + + Dataset rowDatasetWithRecordKeysAndPartitionPath; + if (keyGeneratorClass.equals(NonPartitionedExtractor.class.getName())) { + // for non partitioned, set partition path to empty. + rowDatasetWithRecordKeysAndPartitionPath = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, functions.col(recordKeyFields)) + .withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, functions.lit("").cast(DataTypes.StringType)); + } else if (keyGeneratorClass.equals(SimpleKeyGenerator.class.getName()) + || (keyGeneratorClass.equals(ComplexKeyGenerator.class.getName()) && !recordKeyFields.contains(",") && !partitionPathFields.contains(",") + && (!partitionPathFields.contains("timestamp")))) { // incase of ComplexKeyGen, check partition path type. + // simple fields for both record key and partition path: can directly use withColumn + String partitionPathField = keyGeneratorClass.equals(SimpleKeyGenerator.class.getName()) ? partitionPathFields : + partitionPathFields.substring(partitionPathFields.indexOf(":") + 1); + rowDatasetWithRecordKeysAndPartitionPath = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, functions.col(recordKeyFields).cast(DataTypes.StringType)) + .withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, functions.col(partitionPathField).cast(DataTypes.StringType)); + } else { + // use udf + String tableName = properties.getString(HoodieWriteConfig.TBL_NAME.key()); + String recordKeyUdfFn = RECORD_KEY_UDF_FN + tableName; + String partitionPathUdfFn = PARTITION_PATH_UDF_FN + tableName; + sqlContext.udf().register(recordKeyUdfFn, (UDF1) keyGenerator::getRecordKey, DataTypes.StringType); + sqlContext.udf().register(partitionPathUdfFn, (UDF1) keyGenerator::getPartitionPath, DataTypes.StringType); + + final Dataset rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD, + callUDF(recordKeyUdfFn, org.apache.spark.sql.functions.struct( + JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq()))); + rowDatasetWithRecordKeysAndPartitionPath = + rowDatasetWithRecordKeys.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD, + callUDF(partitionPathUdfFn, + org.apache.spark.sql.functions.struct( + JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq()))); + } // Add other empty hoodie fields which will be populated before writing to parquet. Dataset rowDatasetWithHoodieColumns = rowDatasetWithRecordKeysAndPartitionPath.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD, - functions.lit("").cast(DataTypes.StringType)) + functions.lit("").cast(DataTypes.StringType)) .withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, functions.lit("").cast(DataTypes.StringType)) .withColumn(HoodieRecord.FILENAME_METADATA_FIELD, @@ -106,7 +129,7 @@ public static Dataset prepareHoodieDatasetForBulkInsert(SQLContext sqlConte Dataset processedDf = rowDatasetWithHoodieColumns; if (dropPartitionColumns) { String partitionColumns = String.join(",", keyGenerator.getPartitionPathFields()); - for (String partitionField: keyGenerator.getPartitionPathFields()) { + for (String partitionField : keyGenerator.getPartitionPathFields()) { originalFields.remove(new Column(partitionField)); } processedDf = rowDatasetWithHoodieColumns.drop(partitionColumns); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java index 823de99fc3590..9a793c4227936 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -25,8 +25,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.io.storage.row.HoodieRowCreateHandleWithoutMetaFields; import org.apache.hudi.io.storage.row.HoodieRowCreateHandle; +import org.apache.hudi.io.storage.row.HoodieRowCreateHandleWithoutMetaFields; import org.apache.hudi.keygen.BuiltinKeyGenerator; import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -123,8 +123,7 @@ public void write(InternalRow record) throws IOException { try { String partitionPath = null; if (populateMetaFields) { // usual path where meta fields are pre populated in prep step. - partitionPath = record.getUTF8String( - HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString(); + partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS)); } else { // if meta columns are disabled. if (!keyGeneratorOpt.isPresent()) { // NoPartitionerKeyGen partitionPath = ""; 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 0d4c7cf184ddc..36dd07f28a180 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 @@ -26,11 +26,11 @@ import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.Option import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} -import org.apache.hudi.hive.util.ConfigUtils import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} 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.hudi.sync.common.util.ConfigUtils import org.apache.log4j.LogManager import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala index 58c33248234c2..552e3cfc9b9c3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala @@ -19,14 +19,14 @@ package org.apache.hudi +import org.apache.hudi.avro.model.HoodieClusteringGroup import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.withSparkConf -import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.immutable.Map +import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, mapAsJavaMapConverter} object HoodieCLIUtils { @@ -46,4 +46,15 @@ object HoodieCLIUtils { DataSourceUtils.createHoodieClient(jsc, schemaStr, basePath, metaClient.getTableConfig.getTableName, finalParameters.asJava) } + + def extractPartitions(clusteringGroups: Seq[HoodieClusteringGroup]): String = { + var partitionPaths: Seq[String] = Seq.empty + clusteringGroups.foreach(g => + g.getSlices.asScala.foreach(slice => + partitionPaths = partitionPaths :+ slice.getPartitionPath + ) + ) + + partitionPaths.sorted.mkString(",") + } } 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 4423874ab8e8c..da2736e59bdda 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 @@ -212,7 +212,6 @@ object HoodieSparkSqlWriter { (writeStatuses, client) } case WriteOperationType.DELETE_PARTITION => { - val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) if (!tableExists) { throw new HoodieException(s"hoodie table at $basePath does not exist") } @@ -222,6 +221,7 @@ object HoodieSparkSqlWriter { val partitionColsToDelete = parameters(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).split(",") java.util.Arrays.asList(partitionColsToDelete: _*) } else { + val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema) genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect() } // Create a HoodieWriteClient & issue the delete. @@ -564,8 +564,7 @@ object HoodieSparkSqlWriter { throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + " To use row writer please switch to spark 2 or spark 3") } - val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params) - val syncHiveSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) + val syncHiveSuccess = metaSync(sqlContext.sparkSession, writeConfig, basePath, df.schema) (syncHiveSuccess, common.util.Option.ofNullable(instantTime)) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index 7ee8f6ad569b2..76cea362a3b53 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.hudi.AvroConversionUtils +import org.apache.hudi.DataSourceWriteOptions.OPERATION import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.common.config.DFSPropertiesConfiguration import org.apache.hudi.common.model.HoodieTableType @@ -321,6 +322,8 @@ class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) exten } object HoodieCatalogTable { + // The properties should not be used when create table + val needFilterProps: List[String] = List(HoodieTableConfig.DATABASE_NAME.key, HoodieTableConfig.NAME.key, OPERATION.key) def apply(sparkSession: SparkSession, tableIdentifier: TableIdentifier): HoodieCatalogTable = { val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 31fb0ad6cb0cf..131ebebe85a5a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -255,8 +255,7 @@ trait ProvidesHoodieConfig extends Logging { val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf) val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) - // operation can not be overwrite - val options = hoodieCatalogTable.catalogProperties.-(OPERATION.key()) + val options = hoodieCatalogTable.catalogProperties withSparkConf(sparkSession, options) { Map( diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala index 195bf4153c998..9bf1d721525c3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala @@ -26,6 +26,7 @@ import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps import org.apache.spark.sql.hive.HiveClientUtils import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive @@ -130,8 +131,9 @@ object CreateHoodieTableCommand { .copy(table = tableName, database = Some(newDatabaseName)) val partitionColumnNames = hoodieCatalogTable.partitionSchema.map(_.name) - // append pk, preCombineKey, type to the properties of table - val newTblProperties = hoodieCatalogTable.catalogProperties ++ HoodieOptionConfig.extractSqlOptions(properties) + // Remove some properties should not be used;append pk, preCombineKey, type to the properties of table + val newTblProperties = + hoodieCatalogTable.catalogProperties.--(needFilterProps) ++ HoodieOptionConfig.extractSqlOptions(properties) val newTable = table.copy( identifier = newTableIdentifier, storage = newStorage, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala index 954f08ce645c5..c24d0fd992d97 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/DropHoodieTableCommand.scala @@ -21,12 +21,10 @@ import org.apache.hadoop.fs.Path import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieTableType +import org.apache.hudi.sync.common.util.ConfigUtils import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, HoodieCatalogTable} -import org.apache.spark.sql.hive.HiveClientUtils -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive +import org.apache.spark.sql.catalyst.catalog._ import scala.util.control.NonFatal @@ -69,13 +67,13 @@ extends HoodieLeafRunnableCommand { val catalog = sparkSession.sessionState.catalog // Drop table in the catalog - val enableHive = isEnableHive(sparkSession) - if (enableHive) { - dropHiveDataSourceTable(sparkSession, hoodieCatalogTable) + if (HoodieTableType.MERGE_ON_READ == hoodieCatalogTable.tableType && purge) { + val (rtTableOpt, roTableOpt) = getTableRTAndRO(catalog, hoodieCatalogTable) + rtTableOpt.foreach(table => catalog.dropTable(table.identifier, true, false)) + roTableOpt.foreach(table => catalog.dropTable(table.identifier, true, false)) + catalog.dropTable(table.identifier.copy(table = hoodieCatalogTable.tableName), ifExists, purge) } else { - if (catalog.tableExists(tableIdentifier)) { - catalog.dropTable(tableIdentifier, ifExists, purge) - } + catalog.dropTable(table.identifier, ifExists, purge) } // Recursively delete table directories @@ -88,42 +86,33 @@ extends HoodieLeafRunnableCommand { } } - private def dropHiveDataSourceTable( - sparkSession: SparkSession, - hoodieCatalogTable: HoodieCatalogTable): Unit = { - val table = hoodieCatalogTable.table - val dbName = table.identifier.database.get - val tableName = hoodieCatalogTable.tableName - - // check database exists - val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName) - if (!dbExists) { - throw new NoSuchDatabaseException(dbName) - } - - if (HoodieTableType.MERGE_ON_READ == hoodieCatalogTable.tableType && purge) { - val snapshotTableName = tableName + MOR_SNAPSHOT_TABLE_SUFFIX - val roTableName = tableName + MOR_READ_OPTIMIZED_TABLE_SUFFIX - - dropHiveTable(sparkSession, dbName, snapshotTableName) - dropHiveTable(sparkSession, dbName, roTableName) + private def getTableRTAndRO(catalog: SessionCatalog, + hoodieTable: HoodieCatalogTable): (Option[CatalogTable], Option[CatalogTable]) = { + val rtIdt = hoodieTable.table.identifier.copy( + table = s"${hoodieTable.tableName}${MOR_SNAPSHOT_TABLE_SUFFIX}") + val roIdt = hoodieTable.table.identifier.copy( + table = s"${hoodieTable.tableName}${MOR_READ_OPTIMIZED_TABLE_SUFFIX}") + + var rtTableOpt: Option[CatalogTable] = None + var roTableOpt: Option[CatalogTable] = None + if (catalog.tableExists(rtIdt)) { + val rtTable = catalog.getTableMetadata(rtIdt) + if (rtTable.storage.locationUri.equals(hoodieTable.table.storage.locationUri)) { + rtTable.storage.properties.get(ConfigUtils.IS_QUERY_AS_RO_TABLE) match { + case Some(v) if v.equalsIgnoreCase("false") => rtTableOpt = Some(rtTable) + case _ => // do-nothing + } + } } - - dropHiveTable(sparkSession, dbName, tableName, purge) - } - - private def dropHiveTable( - sparkSession: SparkSession, - dbName: String, - tableName: String, - purge: Boolean = false): Unit = { - // check table exists - if (sparkSession.sessionState.catalog.tableExists(new TableIdentifier(tableName, Option(dbName)))) { - val client = HiveClientUtils.newClientForMetadata(sparkSession.sparkContext.conf, - sparkSession.sessionState.newHadoopConf()) - - // drop hive table. - client.dropTable(dbName, tableName, ifExists, purge) + if (catalog.tableExists(roIdt)) { + val roTable = catalog.getTableMetadata(roIdt) + if (roTable.storage.locationUri.equals(hoodieTable.table.storage.locationUri)) { + roTable.storage.properties.get(ConfigUtils.IS_QUERY_AS_RO_TABLE) match { + case Some(v) if v.equalsIgnoreCase("true") => roTableOpt = Some(roTable) + case _ => // do-nothing + } + } } + (rtTableOpt, roTableOpt) } } 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 5b513f7500c10..57aff092b7429 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 @@ -19,11 +19,9 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableMetaClient - -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE} import org.apache.spark.sql.hudi.command.procedures.{HoodieProcedureUtils, RunCompactionProcedure} -import org.apache.spark.sql.types.StringType import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.unsafe.types.UTF8String @@ -50,10 +48,5 @@ case class CompactionHoodiePathCommand(path: String, RunCompactionProcedure.builder.get().build.call(procedureArgs) } - override val output: Seq[Attribute] = { - operation match { - case RUN => Seq.empty - case SCHEDULE => Seq(AttributeReference("instant", StringType, nullable = false)()) - } - } + override val output: Seq[Attribute] = RunCompactionProcedure.builder.get().build.outputType.toAttributes } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala index 5e362314c2df7..adaaeae9e55c9 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodieTableCommand.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.hudi.command import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.CompactionOperation import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.getTableLocation -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.hudi.command.procedures.RunCompactionProcedure import org.apache.spark.sql.{Row, SparkSession} @Deprecated @@ -35,10 +35,5 @@ case class CompactionHoodieTableCommand(table: CatalogTable, CompactionHoodiePathCommand(basePath, operation, instantTimestamp).run(sparkSession) } - override val output: Seq[Attribute] = { - operation match { - case RUN => Seq.empty - case SCHEDULE => Seq(AttributeReference("instant", StringType, nullable = false)()) - } - } + override val output: Seq[Attribute] = RunCompactionProcedure.builder.get().build.outputType.toAttributes } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala index 965724163b96c..95a4ecf7800e6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodiePathCommand.scala @@ -19,10 +19,8 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableMetaClient - -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.hudi.command.procedures.{HoodieProcedureUtils, ShowCompactionProcedure} -import org.apache.spark.sql.types.{IntegerType, StringType} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.unsafe.types.UTF8String @@ -42,11 +40,5 @@ case class CompactionShowHoodiePathCommand(path: String, limit: Int) ShowCompactionProcedure.builder.get().build.call(procedureArgs) } - override val output: Seq[Attribute] = { - Seq( - AttributeReference("instant", StringType, nullable = false)(), - AttributeReference("action", StringType, nullable = false)(), - AttributeReference("size", IntegerType, nullable = false)() - ) - } + override val output: Seq[Attribute] = ShowCompactionProcedure.builder.get().build.outputType.toAttributes } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala index f3f0a8e529be9..afd15d5153db6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.hudi.command import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.getTableLocation -import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.hudi.command.procedures.ShowCompactionProcedure import org.apache.spark.sql.{Row, SparkSession} @Deprecated @@ -32,11 +32,5 @@ case class CompactionShowHoodieTableCommand(table: CatalogTable, limit: Int) CompactionShowHoodiePathCommand(basePath, limit).run(sparkSession) } - override val output: Seq[Attribute] = { - Seq( - AttributeReference("timestamp", StringType, nullable = false)(), - AttributeReference("action", StringType, nullable = false)(), - AttributeReference("size", IntegerType, nullable = false)() - ) - } + override val output: Seq[Attribute] = ShowCompactionProcedure.builder.get().build.outputType.toAttributes } 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 1d2cea10afa7d..733bd67a0b0d7 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 @@ -21,9 +21,10 @@ 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} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HoodieCatalogTable} +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps +import org.apache.hudi.sync.common.util.ConfigUtils import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils @@ -66,9 +67,21 @@ case class CreateHoodieTableAsSelectCommand( // ReOrder the query which move the partition columns to the last of the project list val reOrderedQuery = reOrderPartitionColumn(query, table.partitionColumnNames) - val tableWithSchema = table.copy(schema = reOrderedQuery.schema) + // Remove some properties should not be used + val newStorage = new CatalogStorageFormat( + table.storage.locationUri, + table.storage.inputFormat, + table.storage.outputFormat, + table.storage.serde, + table.storage.compressed, + table.storage.properties.--(needFilterProps)) + val newTable = table.copy( + storage = newStorage, + schema = reOrderedQuery.schema, + properties = table.properties.--(needFilterProps) + ) - val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableWithSchema) + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, newTable) val tablePath = hoodieCatalogTable.tableLocation val hadoopConf = sparkSession.sessionState.newHadoopConf() assert(HoodieSqlCommonUtils.isEmptyPath(tablePath, hadoopConf), @@ -83,11 +96,11 @@ case class CreateHoodieTableAsSelectCommand( val options = Map( 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), + HiveSyncConfig.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(newTable.properties.asJava), DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" ) - val success = InsertIntoHoodieTableCommand.run(sparkSession, tableWithSchema, reOrderedQuery, Map.empty, + val success = InsertIntoHoodieTableCommand.run(sparkSession, newTable, reOrderedQuery, Map.empty, mode == SaveMode.Overwrite, refreshTable = false, extraOptions = options) if (success) { // If write success, create the table in catalog if it has not synced to the diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala index 231d0939cc2e7..b353aebe50ac9 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL} -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.ValidationUtils.checkArgument import org.apache.hudi.common.util.{ClusteringUtils, Option => HOption} @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.datasources.FileStatusCache import org.apache.spark.sql.types._ import java.util.function.Supplier + import scala.collection.JavaConverters._ class RunClusteringProcedure extends BaseProcedure @@ -50,13 +51,15 @@ class RunClusteringProcedure extends BaseProcedure ProcedureParameter.optional(0, "table", DataTypes.StringType, None), ProcedureParameter.optional(1, "path", DataTypes.StringType, None), ProcedureParameter.optional(2, "predicate", DataTypes.StringType, None), - ProcedureParameter.optional(3, "order", DataTypes.StringType, None) + ProcedureParameter.optional(3, "order", DataTypes.StringType, None), + ProcedureParameter.optional(4, "show_involved_partition", DataTypes.BooleanType, false) ) private val OUTPUT_TYPE = new StructType(Array[StructField]( StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), - StructField("partition", DataTypes.StringType, nullable = true, Metadata.empty), - StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty) + StructField("input_group_size", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("involved_partitions", DataTypes.StringType, nullable = true, Metadata.empty) )) def parameters: Array[ProcedureParameter] = PARAMETERS @@ -70,6 +73,7 @@ class RunClusteringProcedure extends BaseProcedure val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) val predicate = getArgValueOrDefault(args, PARAMETERS(2)) val orderColumns = getArgValueOrDefault(args, PARAMETERS(3)) + val showInvolvedPartitions = getArgValueOrDefault(args, PARAMETERS(4)).get.asInstanceOf[Boolean] val basePath: String = getBasePath(tableName, tablePath) val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build @@ -114,7 +118,27 @@ class RunClusteringProcedure extends BaseProcedure pendingClustering.foreach(client.cluster(_, true)) logInfo(s"Finish clustering all the instants: ${pendingClustering.mkString(",")}," + s" time cost: ${System.currentTimeMillis() - startTs}ms.") - Seq.empty[Row] + + val clusteringInstants = metaClient.reloadActiveTimeline().getInstants.iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION && pendingClustering.contains(p.getTimestamp)) + .toSeq + .sortBy(f => f.getTimestamp) + .reverse + + val clusteringPlans = clusteringInstants.map(instant => + ClusteringUtils.getClusteringPlan(metaClient, instant) + ) + + if (showInvolvedPartitions) { + clusteringPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + p.get().getLeft.getState.name(), HoodieCLIUtils.extractPartitions(p.get().getRight.getInputGroups.asScala)) + } + } else { + clusteringPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), "*") + } + } } override def build: Procedure = new RunClusteringProcedure() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala index 9bca33f3882d4..3e5a7e29e4022 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunCompactionProcedure.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.common.model.HoodieCommitMetadata import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} -import org.apache.hudi.common.util.{HoodieTimer, Option => HOption} +import org.apache.hudi.common.util.{CompactionUtils, HoodieTimer, Option => HOption} import org.apache.hudi.exception.HoodieException import org.apache.hudi.{HoodieCLIUtils, SparkAdapterSupport} - import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.types._ @@ -47,7 +46,9 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp ) private val OUTPUT_TYPE = new StructType(Array[StructField]( - StructField("instant", DataTypes.StringType, nullable = true, Metadata.empty) + StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("operation_size", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty) )) def parameters: Array[ProcedureParameter] = PARAMETERS @@ -66,13 +67,12 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, basePath, Map.empty) + var willCompactionInstants: Seq[String] = Seq.empty operation match { case "schedule" => val instantTime = instantTimestamp.map(_.toString).getOrElse(HoodieActiveTimeline.createNewInstantTime) if (client.scheduleCompactionAtInstant(instantTime, HOption.empty[java.util.Map[String, String]])) { - Seq(Row(instantTime)) - } else { - Seq.empty[Row] + willCompactionInstants = Seq(instantTime) } case "run" => // Do compaction @@ -81,7 +81,7 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp .filter(p => p.getAction == HoodieTimeline.COMPACTION_ACTION) .map(_.getTimestamp) .toSeq.sortBy(f => f) - val willCompactionInstants = if (instantTimestamp.isEmpty) { + willCompactionInstants = if (instantTimestamp.isEmpty) { if (pendingCompactionInstants.nonEmpty) { pendingCompactionInstants } else { // If there are no pending compaction, schedule to generate one. @@ -102,9 +102,9 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp s"$basePath, Available pending compaction instants are: ${pendingCompactionInstants.mkString(",")} ") } } + if (willCompactionInstants.isEmpty) { logInfo(s"No need to compaction on $basePath") - Seq.empty[Row] } else { logInfo(s"Run compaction at instants: [${willCompactionInstants.mkString(",")}] on $basePath") val timer = new HoodieTimer @@ -116,10 +116,21 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp } logInfo(s"Finish Run compaction at instants: [${willCompactionInstants.mkString(",")}]," + s" spend: ${timer.endTimer()}ms") - Seq.empty[Row] } case _ => throw new UnsupportedOperationException(s"Unsupported compaction operation: $operation") } + + val compactionInstants = metaClient.reloadActiveTimeline().getInstants.iterator().asScala + .filter(instant => willCompactionInstants.contains(instant.getTimestamp)) + .toSeq + .sortBy(p => p.getTimestamp) + .reverse + + compactionInstants.map(instant => + (instant, CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp)) + ).map { case (instant, plan) => + Row(instant.getTimestamp, plan.getOperations.size(), instant.getState.name()) + } } private def handleResponse(metadata: HoodieCommitMetadata): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala index a9d808217c0a9..092610119e606 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala @@ -17,26 +17,31 @@ package org.apache.spark.sql.hudi.command.procedures -import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.{HoodieCLIUtils, SparkAdapterSupport} import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.util.ClusteringUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.types._ import java.util.function.Supplier + import scala.collection.JavaConverters._ class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with SparkAdapterSupport with Logging { private val PARAMETERS = Array[ProcedureParameter]( ProcedureParameter.optional(0, "table", DataTypes.StringType, None), ProcedureParameter.optional(1, "path", DataTypes.StringType, None), - ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20) + ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20), + ProcedureParameter.optional(3, "show_involved_partition", DataTypes.BooleanType, false) ) private val OUTPUT_TYPE = new StructType(Array[StructField]( StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), - StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty) + StructField("input_group_size", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("involved_partitions", DataTypes.StringType, nullable = true, Metadata.empty) )) def parameters: Array[ProcedureParameter] = PARAMETERS @@ -49,12 +54,32 @@ class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with S val tableName = getArgValueOrDefault(args, PARAMETERS(0)) val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int] + val showInvolvedPartitions = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[Boolean] val basePath: String = getBasePath(tableName, tablePath) val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build - ClusteringUtils.getAllPendingClusteringPlans(metaClient).iterator().asScala.map { p => - Row(p.getLeft.getTimestamp, p.getRight.getInputGroups.size()) - }.toSeq.take(limit) + val clusteringInstants = metaClient.getActiveTimeline.getInstants.iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + .sortBy(f => f.getTimestamp) + .reverse + .take(limit) + + val clusteringPlans = clusteringInstants.map(instant => + ClusteringUtils.getClusteringPlan(metaClient, instant) + ) + + if (showInvolvedPartitions) { + clusteringPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + p.get().getLeft.getState.name(), HoodieCLIUtils.extractPartitions(p.get().getRight.getInputGroups.asScala)) + } + } else { + clusteringPlans.map { p => + Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + p.get().getLeft.getState.name(), "*") + } + } } override def build: Procedure = new ShowClusteringProcedure() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala index d484d65323447..7a7bb2cf9d996 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala @@ -44,8 +44,8 @@ class ShowCompactionProcedure extends BaseProcedure with ProcedureBuilder with S private val OUTPUT_TYPE = new StructType(Array[StructField]( StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), - StructField("action", DataTypes.StringType, nullable = true, Metadata.empty), - StructField("size", DataTypes.IntegerType, nullable = true, Metadata.empty) + StructField("operation_size", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("state", DataTypes.StringType, nullable = true, Metadata.empty) )) def parameters: Array[ProcedureParameter] = PARAMETERS @@ -64,17 +64,17 @@ class ShowCompactionProcedure extends BaseProcedure with ProcedureBuilder with S assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ, s"Cannot show compaction on a Non Merge On Read table.") - val timeLine = metaClient.getActiveTimeline - val compactionInstants = timeLine.getInstants.iterator().asScala + val compactionInstants = metaClient.getActiveTimeline.getInstants.iterator().asScala .filter(p => p.getAction == HoodieTimeline.COMPACTION_ACTION) .toSeq .sortBy(f => f.getTimestamp) .reverse .take(limit) - val compactionPlans = compactionInstants.map(instant => - (instant, CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp))) - compactionPlans.map { case (instant, plan) => - Row(instant.getTimestamp, instant.getAction, plan.getOperations.size()) + + compactionInstants.map(instant => + (instant, CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp)) + ).map { case (instant, plan) => + Row(instant.getTimestamp, plan.getOperations.size(), instant.getState.name()) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java index 9185d09aad1d0..6b617ca208185 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieDatasetBulkInsertHelper.java @@ -24,6 +24,9 @@ import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.execution.bulkinsert.NonSortPartitionerWithRows; +import org.apache.hudi.keygen.ComplexKeyGenerator; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.testutils.DataSourceTestUtils; import org.apache.hudi.testutils.HoodieClientTestBase; @@ -94,20 +97,36 @@ private void init() throws IOException { public void testBulkInsertHelperConcurrently() { IntStream.range(0, 2).parallel().forEach(i -> { if (i % 2 == 0) { - testBulkInsertHelperFor("_row_key"); + testBulkInsertHelperFor(SimpleKeyGenerator.class.getName(), "_row_key"); } else { - testBulkInsertHelperFor("ts"); + testBulkInsertHelperFor(SimpleKeyGenerator.class.getName(), "ts"); } }); } - @Test - public void testBulkInsertHelper() { - testBulkInsertHelperFor("_row_key"); + private static Stream provideKeyGenArgs() { + return Stream.of( + Arguments.of(SimpleKeyGenerator.class.getName()), + Arguments.of(ComplexKeyGenerator.class.getName()), + Arguments.of(NonpartitionedKeyGenerator.class.getName())); } - private void testBulkInsertHelperFor(String recordKey) { - HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(getPropsAllSet(recordKey)).combineInput(false, false).build(); + @ParameterizedTest + @MethodSource("provideKeyGenArgs") + public void testBulkInsertHelper(String keyGenClass) { + testBulkInsertHelperFor(keyGenClass, "_row_key"); + } + + private void testBulkInsertHelperFor(String keyGenClass, String recordKey) { + Map props = null; + if (keyGenClass.equals(SimpleKeyGenerator.class.getName())) { + props = getPropsAllSet(recordKey); + } else if (keyGenClass.equals(ComplexKeyGenerator.class.getName())) { + props = getPropsForComplexKeyGen(recordKey); + } else { // NonPartitioned key gen + props = getPropsForNonPartitionedKeyGen(recordKey); + } + HoodieWriteConfig config = getConfigBuilder(schemaStr).withProps(props).combineInput(false, false).build(); List rows = DataSourceTestUtils.generateRandomRows(10); Dataset dataset = sqlContext.createDataFrame(rows, structType); Dataset result = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, config, dataset, "testStructName", @@ -121,9 +140,10 @@ private void testBulkInsertHelperFor(String recordKey) { assertTrue(resultSchema.fieldIndex(entry.getKey()) == entry.getValue()); } + boolean isNonPartitioned = keyGenClass.equals(NonpartitionedKeyGenerator.class.getName()); result.toJavaRDD().foreach(entry -> { assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.RECORD_KEY_METADATA_FIELD)).equals(entry.getAs(recordKey).toString())); - assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(entry.getAs("partition"))); + assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).equals(isNonPartitioned ? "" : entry.getAs("partition"))); assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)).equals("")); assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).equals("")); assertTrue(entry.get(resultSchema.fieldIndex(HoodieRecord.FILENAME_METADATA_FIELD)).equals("")); @@ -253,6 +273,23 @@ private Map getProps(String recordKey, boolean setAll, boolean s return props; } + private Map getPropsForComplexKeyGen(String recordKey) { + Map props = new HashMap<>(); + props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), ComplexKeyGenerator.class.getName()); + props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKey); + props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "simple:partition"); + props.put(HoodieWriteConfig.TBL_NAME.key(), recordKey + "_table"); + return props; + } + + private Map getPropsForNonPartitionedKeyGen(String recordKey) { + Map props = new HashMap<>(); + props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), NonpartitionedKeyGenerator.class.getName()); + props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKey); + props.put(HoodieWriteConfig.TBL_NAME.key(), recordKey + "_table"); + return props; + } + @Test public void testNoPropsSet() { HoodieWriteConfig config = getConfigBuilder(schemaStr).build(); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java deleted file mode 100644 index 4491a74fa62ba..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/index/bucket/TestBucketIdentifier.java +++ /dev/null @@ -1,67 +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.index.bucket; - -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.keygen.KeyGenUtils; -import org.apache.hudi.testutils.KeyGeneratorTestUtilities; - -import org.apache.avro.generic.GenericRecord; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.List; - -public class TestBucketIdentifier { - - @Test - public void testBucketFileId() { - for (int i = 0; i < 1000; i++) { - String bucketId = BucketIdentifier.bucketIdStr(i); - String fileId = BucketIdentifier.newBucketFileIdPrefix(bucketId); - assert BucketIdentifier.bucketIdFromFileId(fileId) == i; - } - } - - @Test - public void testBucketIdWithSimpleRecordKey() { - String recordKeyField = "_row_key"; - String indexKeyField = "_row_key"; - GenericRecord record = KeyGeneratorTestUtilities.getRecord(); - HoodieRecord hoodieRecord = new HoodieAvroRecord( - new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); - int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); - assert bucketId == BucketIdentifier.getBucketId( - Arrays.asList(record.get(indexKeyField).toString()), 8); - } - - @Test - public void testBucketIdWithComplexRecordKey() { - List recordKeyField = Arrays.asList("_row_key","ts_ms"); - String indexKeyField = "_row_key"; - GenericRecord record = KeyGeneratorTestUtilities.getRecord(); - HoodieRecord hoodieRecord = new HoodieAvroRecord( - new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField, false), ""), null); - int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8); - assert bucketId == BucketIdentifier.getBucketId( - Arrays.asList(record.get(indexKeyField).toString()), 8); - } -} diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java index 735277d959ee4..6719c2a3d6d23 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java @@ -83,7 +83,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java index 3bd6a60c4c1ea..f6c4c8a8b58cd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteRecordGenerator.java @@ -68,7 +68,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { GlobalDeleteKeyGenerator keyGenerator = new GlobalDeleteKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -78,7 +78,7 @@ public void testHappyFlow() { HoodieKey key = keyGenerator.getKey(record); Assertions.assertEquals(key.getRecordKey(), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(key.getPartitionPath(), ""); - keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType); + keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType); Row row = KeyGeneratorTestUtilities.getRow(record); Assertions.assertEquals(keyGenerator.getRecordKey(row), "_row_key:key1,pii_col:pi"); Assertions.assertEquals(keyGenerator.getPartitionPath(row), ""); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java index 297b077794d56..75d9b7da74bc8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java @@ -94,7 +94,7 @@ public void testNullPartitionPathFields() { public void testWrongRecordKeyField() { NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java index 7dea9e414e693..17cff3505ebef 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java @@ -100,7 +100,7 @@ public void testNullRecordKeyFields() { public void testWrongRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getWrongRecordKeyFieldProps()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test @@ -116,7 +116,7 @@ public void testWrongPartitionPathField() { public void testComplexRecordKeyField() { SimpleKeyGenerator keyGenerator = new SimpleKeyGenerator(getComplexRecordKeyProp()); Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.getRecordKey(getRecord())); - Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldPositionMapIfNeeded(KeyGeneratorTestUtilities.structType)); + Assertions.assertThrows(HoodieKeyException.class, () -> keyGenerator.buildFieldSchemaInfoIfNeeded(KeyGeneratorTestUtilities.structType)); } @Test 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 111a46261c769..339dbb5c715ef 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 @@ -20,7 +20,6 @@ package org.apache.hudi import java.io.IOException import java.time.Instant import java.util.{Collections, Date, UUID} - import org.apache.commons.io.FileUtils import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.client.SparkRDDWriteClient diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSqlStatement.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSqlStatement.scala index c451b51ef77c6..f8a9cf5fb060f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSqlStatement.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSqlStatement.scala @@ -18,9 +18,9 @@ package org.apache.hudi.functional import org.apache.hudi.common.util.FileIOUtils -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase -class TestSqlStatement extends TestHoodieSqlBase { +class TestSqlStatement extends HoodieSparkSqlTestBase { val STATE_INIT = 0 val STATE_SKIP_COMMENT = 1 val STATE_FINISH_COMMENT = 2 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchmark.scala index d84fad4f2493c..273303fdae63d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/SpaceCurveOptimizeBenchmark.scala @@ -23,7 +23,7 @@ import org.apache.hudi.ColumnStatsIndexHelper.buildColumnStatsTableFor import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy import org.apache.hudi.sort.SpaceCurveSortingHelper import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase import org.apache.spark.sql.types.{IntegerType, StructField} import org.junit.jupiter.api.{Disabled, Tag, Test} @@ -31,7 +31,7 @@ import scala.collection.JavaConversions._ import scala.util.Random @Tag("functional") -object SpaceCurveOptimizeBenchmark extends TestHoodieSqlBase { +object SpaceCurveOptimizeBenchmark extends HoodieSparkSqlTestBase { def evalSkippingPercent(tableName: String, co1: String, co2: String, value1: Int, value2: Int): Unit= { val sourceTableDF = spark.sql(s"select * from ${tableName}") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala similarity index 98% rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala index d1f373db99e51..68fc6d7c41d89 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala @@ -31,7 +31,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Tag} import java.io.File import java.util.TimeZone -class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll { +class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll { org.apache.log4j.Logger.getRootLogger.setLevel(Level.WARN) private lazy val sparkWareHouse = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala index 0f2cb547c2fe9..6d29ea3f4a13e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.types.{LongType, StructField, StructType} -class TestAlterTable extends TestHoodieSqlBase { +class TestAlterTable extends HoodieSparkSqlTestBase { test("Test Alter Table") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala index ecbbadeeb9a28..677f8632a7143 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala @@ -23,7 +23,7 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} import org.apache.spark.sql.SaveMode -class TestAlterTableDropPartition extends TestHoodieSqlBase { +class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { test("Drop non-partitioned table") { val tableName = generateTableName diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala index 20238a6e4318d..0ef89fc5b9fe3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCompactionTable.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hudi -class TestCompactionTable extends TestHoodieSqlBase { +class TestCompactionTable extends HoodieSparkSqlTestBase { test("Test compaction table") { withTempDir {tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 6b8efb84e32f1..5435aad05e88a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ +import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.config.HoodieWriteConfig @@ -28,9 +29,11 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.types._ +import org.junit.jupiter.api.Assertions.assertFalse + import scala.collection.JavaConverters._ -class TestCreateTable extends TestHoodieSqlBase { +class TestCreateTable extends HoodieSparkSqlTestBase { test("Test Create Managed Hoodie Table") { val databaseName = "hudi_database" @@ -48,8 +51,11 @@ class TestCreateTable extends TestHoodieSqlBase { | ts long | ) using hudi | tblproperties ( + | hoodie.database.name = "databaseName", + | hoodie.table.name = "tableName", | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | hoodie.datasource.write.operation = 'upsert' | ) """.stripMargin) val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) @@ -64,6 +70,9 @@ class TestCreateTable extends TestHoodieSqlBase { StructField("price", DoubleType), StructField("ts", LongType)) )(table.schema.fields) + assertFalse(table.properties.contains(HoodieTableConfig.DATABASE_NAME.key())) + assertFalse(table.properties.contains(HoodieTableConfig.NAME.key())) + assertFalse(table.properties.contains(OPERATION.key())) val tablePath = table.storage.properties("path") val metaClient = HoodieTableMetaClient.builder() @@ -72,6 +81,10 @@ class TestCreateTable extends TestHoodieSqlBase { .build() val tableConfig = metaClient.getTableConfig assertResult(databaseName)(tableConfig.getDatabaseName) + assertResult(tableName)(tableConfig.getTableName) + assertFalse(tableConfig.contains(OPERATION.key())) + + spark.sql("use default") } test("Test Create Hoodie Table With Options") { @@ -87,8 +100,11 @@ class TestCreateTable extends TestHoodieSqlBase { | ) using hudi | partitioned by (dt) | options ( + | hoodie.database.name = "databaseName", + | hoodie.table.name = "tableName", | primaryKey = 'id', - | preCombineField = 'ts' + | preCombineField = 'ts', + | hoodie.datasource.write.operation = 'upsert' | ) """.stripMargin) val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) @@ -107,6 +123,9 @@ class TestCreateTable extends TestHoodieSqlBase { StructField("ts", LongType), StructField("dt", StringType)) )(table.schema.fields) + assertFalse(table.properties.contains(HoodieTableConfig.DATABASE_NAME.key())) + assertFalse(table.properties.contains(HoodieTableConfig.NAME.key())) + assertFalse(table.properties.contains(OPERATION.key())) val tablePath = table.storage.properties("path") val metaClient = HoodieTableMetaClient.builder() @@ -119,6 +138,9 @@ class TestCreateTable extends TestHoodieSqlBase { assertResult("id")(tableConfig(HoodieTableConfig.RECORDKEY_FIELDS.key)) assertResult("ts")(tableConfig(HoodieTableConfig.PRECOMBINE_FIELD.key)) assertResult(classOf[ComplexKeyGenerator].getCanonicalName)(tableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) + assertResult("default")(tableConfig(HoodieTableConfig.DATABASE_NAME.key())) + assertResult(tableName)(tableConfig(HoodieTableConfig.NAME.key())) + assertFalse(tableConfig.contains(OPERATION.key())) } test("Test Create External Hoodie Table") { @@ -360,6 +382,84 @@ class TestCreateTable extends TestHoodieSqlBase { } } + test("Test Create Table As Select With Tblproperties For Filter Props") { + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName using hudi + | partitioned by (dt) + | tblproperties( + | hoodie.database.name = "databaseName", + | hoodie.table.name = "tableName", + | primaryKey = 'id', + | preCombineField = 'ts', + | hoodie.datasource.write.operation = 'upsert', + | type = '$tableType' + | ) + | AS + | select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt, 1000 as ts + """.stripMargin + ) + checkAnswer(s"select id, name, price, dt from $tableName")( + Seq(1, "a1", 10, "2021-04-01") + ) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assertFalse(table.properties.contains(HoodieTableConfig.DATABASE_NAME.key())) + assertFalse(table.properties.contains(HoodieTableConfig.NAME.key())) + assertFalse(table.properties.contains(OPERATION.key())) + + val tablePath = table.storage.properties("path") + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(tablePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + val tableConfig = metaClient.getTableConfig.getProps.asScala.toMap + assertResult("default")(tableConfig(HoodieTableConfig.DATABASE_NAME.key())) + assertResult(tableName)(tableConfig(HoodieTableConfig.NAME.key())) + assertFalse(tableConfig.contains(OPERATION.key())) + } + } + + test("Test Create Table As Select With Options For Filter Props") { + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName using hudi + | partitioned by (dt) + | options( + | hoodie.database.name = "databaseName", + | hoodie.table.name = "tableName", + | primaryKey = 'id', + | preCombineField = 'ts', + | hoodie.datasource.write.operation = 'upsert', + | type = '$tableType' + | ) + | AS + | select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt, 1000 as ts + """.stripMargin + ) + checkAnswer(s"select id, name, price, dt from $tableName")( + Seq(1, "a1", 10, "2021-04-01") + ) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assertFalse(table.properties.contains(HoodieTableConfig.DATABASE_NAME.key())) + assertFalse(table.properties.contains(HoodieTableConfig.NAME.key())) + assertFalse(table.properties.contains(OPERATION.key())) + + val tablePath = table.storage.properties("path") + val metaClient = HoodieTableMetaClient.builder() + .setBasePath(tablePath) + .setConf(spark.sessionState.newHadoopConf()) + .build() + val tableConfig = metaClient.getTableConfig.getProps.asScala.toMap + assertResult("default")(tableConfig(HoodieTableConfig.DATABASE_NAME.key())) + assertResult(tableName)(tableConfig(HoodieTableConfig.NAME.key())) + assertFalse(tableConfig.contains(OPERATION.key())) + } + } + test("Test Create Table As Select when 'spark.sql.datetime.java8API.enabled' enables") { try { // enable spark.sql.datetime.java8API.enabled @@ -641,4 +741,26 @@ class TestCreateTable extends TestHoodieSqlBase { |""".stripMargin ) } + + if (HoodieSparkUtils.gteqSpark3_2) { + test("Test create table with comment") { + val tableName = generateTableName + spark.sql( + s""" + | create table $tableName ( + | id int, + | name string, + | price double, + | ts long + | ) using hudi + | comment "This is a simple hudi table" + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + val shown = spark.sql(s"show create table $tableName").head.getString(0) + assertResult(true)(shown.contains("COMMENT 'This is a simple hudi table'")) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala index b2e888a5f3140..4c7c6269667ab 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDeleteTable.scala @@ -22,7 +22,7 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.SimpleKeyGenerator import org.apache.spark.sql.SaveMode -class TestDeleteTable extends TestHoodieSqlBase { +class TestDeleteTable extends HoodieSparkSqlTestBase { test("Test Delete Table") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala index c53eb9127c887..174835cbac0bf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.hudi -class TestDropTable extends TestHoodieSqlBase { +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.SessionCatalog + +class TestDropTable extends HoodieSparkSqlTestBase { test("Test Drop Table") { withTempDir { tmp => @@ -72,4 +75,167 @@ class TestDropTable extends TestHoodieSqlBase { } } } + + test("Test Drop RO & RT table by purging base table.") { + withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + spark.sql( + s""" + |create table ${tableName}_ro using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"), + Map("hoodie.query.as.ro.table" -> "true")) + + spark.sql( + s""" + |create table ${tableName}_rt using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"), + Map("hoodie.query.as.ro.table" -> "false")) + + spark.sql(s"drop table ${tableName} purge") + checkAnswer("show tables")() + } + } + + test("Test Drop RO & RT table by one by one.") { + withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + spark.sql( + s""" + |create table ${tableName}_ro using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"), + Map("hoodie.query.as.ro.table" -> "true")) + + spark.sql( + s""" + |create table ${tableName}_rt using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"), + Map("hoodie.query.as.ro.table" -> "false")) + + spark.sql(s"drop table ${tableName}_ro") + checkAnswer("show tables")( + Seq("default", tableName, false), Seq("default", s"${tableName}_rt", false)) + + spark.sql(s"drop table ${tableName}_rt") + checkAnswer("show tables")(Seq("default", tableName, false)) + + spark.sql(s"drop table ${tableName}") + checkAnswer("show tables")() + } + } + + test("Test Drop RO table with purge") { + withTempDir { tmp => + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + spark.sql( + s""" + |create table ${tableName}_ro using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_ro"), + Map("hoodie.query.as.ro.table" -> "true")) + + spark.sql( + s""" + |create table ${tableName}_rt using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + alterSerdeProperties(spark.sessionState.catalog, TableIdentifier(s"${tableName}_rt"), + Map("hoodie.query.as.ro.table" -> "false")) + + spark.sql(s"drop table ${tableName}_ro purge") + checkAnswer("show tables")() + } + } + + private def alterSerdeProperties(sessionCatalog: SessionCatalog, tableIdt: TableIdentifier, + newProperties: Map[String, String]): Unit = { + val catalogTable = spark.sessionState.catalog.getTableMetadata(tableIdt) + val storage = catalogTable.storage + val storageProperties = storage.properties ++ newProperties + val newCatalogTable = catalogTable.copy(storage = storage.copy(properties = storageProperties)) + sessionCatalog.alterTable(newCatalogTable) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala index 4c0c60385104b..14c2245d5be36 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -19,27 +19,13 @@ package org.apache.spark.sql.hudi import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, OverwriteWithLatestAvroPayload} import org.apache.hudi.common.table.HoodieTableConfig -import org.apache.hudi.testutils.HoodieClientTestBase - -import org.apache.spark.sql.SparkSession +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.spark.sql.types._ - import org.junit.jupiter.api.Assertions.assertTrue -import org.junit.jupiter.api.{BeforeEach, Test} - +import org.junit.jupiter.api.Test import org.scalatest.Matchers.intercept -class TestHoodieOptionConfig extends HoodieClientTestBase { - - var spark: SparkSession = _ - - /** - * Setup method running before each test. - */ - @BeforeEach override def setUp() { - initSparkContexts() - spark = sqlContext.sparkSession - } +class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness { @Test def testWithDefaultSqlOptions(): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 3141208db121e..ab75ef563f229 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SaveMode import java.io.File -class TestInsertTable extends TestHoodieSqlBase { +class TestInsertTable extends HoodieSparkSqlTestBase { test("Test Insert Into") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala index 5139825f9428f..232b6bbb511c5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoLogOnlyTable.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.testutils.DataSourceTestUtils -class TestMergeIntoLogOnlyTable extends TestHoodieSqlBase { +class TestMergeIntoLogOnlyTable extends HoodieSparkSqlTestBase { test("Test Query Log Only MOR Table") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala index 28dee88e1f61e..992a442f4fda3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.{DataSourceReadOptions, HoodieDataSourceHelpers} import org.apache.hudi.common.fs.FSUtils -class TestMergeIntoTable extends TestHoodieSqlBase { +class TestMergeIntoTable extends HoodieSparkSqlTestBase { test("Test MergeInto Basic") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala index 5041a543168bf..e162368dacc72 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestMergeIntoTable2.scala @@ -21,7 +21,7 @@ import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.spark.sql.Row -class TestMergeIntoTable2 extends TestHoodieSqlBase { +class TestMergeIntoTable2 extends HoodieSparkSqlTestBase { test("Test MergeInto for MOR table 2") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala index 2524d04ec81fb..1af7a162be185 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestPartialUpdateForMergeInto.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hudi -class TestPartialUpdateForMergeInto extends TestHoodieSqlBase { +class TestPartialUpdateForMergeInto extends HoodieSparkSqlTestBase { test("Test Partial Update") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala index 868bfc43d57f1..369f3b341adce 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestShowPartitions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.spark.sql.Row -class TestShowPartitions extends TestHoodieSqlBase { +class TestShowPartitions extends HoodieSparkSqlTestBase { test("Test Show Non Partitioned Table's Partitions") { val tableName = generateTableName diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala index 54163635984bf..15fed579bba41 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSpark3DDL.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ -class TestSpark3DDL extends TestHoodieSqlBase { +class TestSpark3DDL extends HoodieSparkSqlTestBase { def createTestResult(tableName: String): Array[Row] = { spark.sql(s"select * from ${tableName} order by id") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala index 1a8ac0e645899..ac3c49efdd713 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestSqlConf.scala @@ -28,7 +28,7 @@ import java.nio.file.{Files, Paths} import org.scalatest.BeforeAndAfter -class TestSqlConf extends TestHoodieSqlBase with BeforeAndAfter { +class TestSqlConf extends HoodieSparkSqlTestBase with BeforeAndAfter { def setEnv(key: String, value: String): String = { val field = System.getenv().getClass.getDeclaredField("m") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala index 471ebd6107dcc..ce0f17c3f569c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.HoodieTableMetaClient -class TestTimeTravelTable extends TestHoodieSqlBase { +class TestTimeTravelTable extends HoodieSparkSqlTestBase { test("Test Insert and Update Record with time travel") { if (HoodieSparkUtils.gteqSpark3_2) { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala index a61d0f822cf45..5dd243079efb7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala @@ -23,7 +23,7 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} import org.apache.spark.sql.SaveMode -class TestTruncateTable extends TestHoodieSqlBase { +class TestTruncateTable extends HoodieSparkSqlTestBase { test("Test Truncate non-partitioned Table") { Seq("cow", "mor").foreach { tableType => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 57c4a972960a9..8c709ab37a6e3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hudi -class TestUpdateTable extends TestHoodieSqlBase { +class TestUpdateTable extends HoodieSparkSqlTestBase { test("Test Update Table") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala index 87814763bf4d3..668fb544934dd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala @@ -21,13 +21,13 @@ import com.google.common.collect.ImmutableList import org.apache.hudi.HoodieSparkUtils import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{CallCommand, NamedArgument, PositionalArgument} -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase import org.apache.spark.sql.types.{DataType, DataTypes} import java.math.BigDecimal import scala.collection.JavaConverters -class TestCallCommandParser extends TestHoodieSqlBase { +class TestCallCommandParser extends HoodieSparkSqlTestBase { private val parser = spark.sessionState.sqlParser test("Test Call Produce with Positional Arguments") { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala index bdf4cbe7ba0ff..f75569a1171f5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.hudi.procedure -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase -class TestCallProcedure extends TestHoodieSqlBase { +class TestCallProcedure extends HoodieSparkSqlTestBase { test("Test Call show_commits Procedure") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala index 6214117233467..df4d8c90e2e6f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala @@ -20,15 +20,14 @@ package org.apache.spark.sql.hudi.procedure import org.apache.hadoop.fs.Path -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline} import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers} - -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase import scala.collection.JavaConverters.asScalaIteratorConverter -class TestClusteringProcedure extends TestHoodieSqlBase { +class TestClusteringProcedure extends HoodieSparkSqlTestBase { test("Test Call run_clustering Procedure By Table") { withTempDir { tmp => @@ -64,28 +63,22 @@ class TestClusteringProcedure extends TestHoodieSqlBase { val secondScheduleInstant = HoodieActiveTimeline.createNewInstantTime client.scheduleClusteringAtInstant(secondScheduleInstant, HOption.empty()) checkAnswer(s"call show_clustering('$tableName')")( - Seq(firstScheduleInstant, 3), - Seq(secondScheduleInstant, 1) + Seq(secondScheduleInstant, 1, HoodieInstant.State.REQUESTED.name(), "*"), + Seq(firstScheduleInstant, 3, HoodieInstant.State.REQUESTED.name(), "*") ) // Do clustering for all clustering plan generated above, and no new clustering // instant will be generated because of there is no commit after the second // clustering plan generated - spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')") + checkAnswer(s"call run_clustering(table => '$tableName', order => 'ts', show_involved_partition => true)")( + Seq(secondScheduleInstant, 1, HoodieInstant.State.COMPLETED.name(), "ts=1003"), + Seq(firstScheduleInstant, 3, HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001,ts=1002") + ) // No new commits val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, secondScheduleInstant)) - checkAnswer(s"select id, name, price, ts from $tableName order by id")( - Seq(1, "a1", 10.0, 1000), - Seq(2, "a2", 10.0, 1001), - Seq(3, "a3", 10.0, 1002), - Seq(4, "a4", 10.0, 1003) - ) - // After clustering there should be no pending clustering. - checkAnswer(s"call show_clustering(table => '$tableName')")() - // Check the number of finished clustering instants val finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) .getInstants @@ -94,10 +87,23 @@ class TestClusteringProcedure extends TestHoodieSqlBase { .toSeq assertResult(2)(finishedClustering.size) + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003) + ) + + // After clustering there should be no pending clustering and all clustering instants should be completed + checkAnswer(s"call show_clustering(table => '$tableName')")( + Seq(secondScheduleInstant, 1, HoodieInstant.State.COMPLETED.name(), "*"), + Seq(firstScheduleInstant, 3, HoodieInstant.State.COMPLETED.name(), "*") + ) + // Do clustering without manual schedule(which will do the schedule if no pending clustering exists) spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") - spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')") + spark.sql(s"call run_clustering(table => '$tableName', order => 'ts', show_involved_partition => true)").show() val thirdClusteringInstant = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) .findInstantsAfter(secondScheduleInstant) @@ -142,7 +148,7 @@ class TestClusteringProcedure extends TestHoodieSqlBase { | location '$basePath' """.stripMargin) - spark.sql(s"call run_clustering(path => '$basePath')") + spark.sql(s"call run_clustering(path => '$basePath')").show() checkAnswer(s"call show_clustering(path => '$basePath')")() spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") @@ -152,18 +158,22 @@ class TestClusteringProcedure extends TestHoodieSqlBase { // Generate the first clustering plan val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty()) - checkAnswer(s"call show_clustering(path => '$basePath')")( - Seq(firstScheduleInstant, 3) + checkAnswer(s"call show_clustering(path => '$basePath', show_involved_partition => true)")( + Seq(firstScheduleInstant, 3, HoodieInstant.State.REQUESTED.name(), "ts=1000,ts=1001,ts=1002") ) // Do clustering for all the clustering plan - spark.sql(s"call run_clustering(path => '$basePath', order => 'ts')") + checkAnswer(s"call run_clustering(path => '$basePath', order => 'ts')")( + Seq(firstScheduleInstant, 3, HoodieInstant.State.COMPLETED.name(), "*") + ) + checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), Seq(2, "a2", 10.0, 1001), Seq(3, "a3", 10.0, 1002) ) + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) - HoodieDataSourceHelpers.hasNewCommits(fs, basePath, firstScheduleInstant) + assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, firstScheduleInstant)) // Check the number of finished clustering instants var finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) @@ -176,7 +186,12 @@ class TestClusteringProcedure extends TestHoodieSqlBase { // Do clustering without manual schedule(which will do the schedule if no pending clustering exists) spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") - spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts >= 1003L')") + val resultA = spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts >= 1003L', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultA.length) + assertResult("ts=1003,ts=1004")(resultA(0)(3)) + checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), Seq(2, "a2", 10.0, 1001), @@ -220,6 +235,8 @@ class TestClusteringProcedure extends TestHoodieSqlBase { val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) // Test partition pruning with single predicate + var resultA: Array[Seq[Any]] = Array.empty + { spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") @@ -230,7 +247,11 @@ class TestClusteringProcedure extends TestHoodieSqlBase { )("Only partition predicates are allowed") // Do clustering table with partition predicate - spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L', order => 'ts')") + resultA = spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L', order => 'ts', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultA.length) + assertResult("ts=1000,ts=1001")(resultA(0)(3)) // There is 1 completed clustering instant val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) @@ -245,9 +266,12 @@ class TestClusteringProcedure extends TestHoodieSqlBase { val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) assertResult(true)(clusteringPlan.isPresent) assertResult(2)(clusteringPlan.get().getInputGroups.size()) + assertResult(resultA(0)(1))(clusteringPlan.get().getInputGroups.size()) - // No pending clustering instant - checkAnswer(s"call show_clustering(table => '$tableName')")() + // All clustering instants are completed + checkAnswer(s"call show_clustering(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001") + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), @@ -257,6 +281,8 @@ class TestClusteringProcedure extends TestHoodieSqlBase { } // Test partition pruning with {@code And} predicates + var resultB: Array[Seq[Any]] = Array.empty + { spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") @@ -267,7 +293,11 @@ class TestClusteringProcedure extends TestHoodieSqlBase { )("Only partition predicates are allowed") // Do clustering table with partition predicate - spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L', order => 'ts')") + resultB = spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L', order => 'ts', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultB.length) + assertResult("ts=1002,ts=1003,ts=1004,ts=1005")(resultB(0)(3)) // There are 2 completed clustering instants val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) @@ -283,8 +313,11 @@ class TestClusteringProcedure extends TestHoodieSqlBase { assertResult(true)(clusteringPlan.isPresent) assertResult(4)(clusteringPlan.get().getInputGroups.size()) - // No pending clustering instant - checkAnswer(s"call show_clustering(table => '$tableName')")() + // All clustering instants are completed + checkAnswer(s"call show_clustering(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001"), + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1002,ts=1003,ts=1004,ts=1005") + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), @@ -297,6 +330,8 @@ class TestClusteringProcedure extends TestHoodieSqlBase { } // Test partition pruning with {@code And}-{@code Or} predicates + var resultC: Array[Seq[Any]] = Array.empty + { spark.sql(s"insert into $tableName values(7, 'a7', 10, 1006)") spark.sql(s"insert into $tableName values(8, 'a8', 10, 1007)") @@ -308,7 +343,11 @@ class TestClusteringProcedure extends TestHoodieSqlBase { )("Only partition predicates are allowed") // Do clustering table with partition predicate - spark.sql(s"call run_clustering(table => '$tableName', predicate => '(ts >= 1006L and ts < 1008L) or ts >= 1009L', order => 'ts')") + resultC = spark.sql(s"call run_clustering(table => '$tableName', predicate => '(ts >= 1006L and ts < 1008L) or ts >= 1009L', order => 'ts', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(resultC.length) + assertResult("ts=1006,ts=1007,ts=1009")(resultC(0)(3)) // There are 3 completed clustering instants val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) @@ -324,8 +363,12 @@ class TestClusteringProcedure extends TestHoodieSqlBase { assertResult(true)(clusteringPlan.isPresent) assertResult(3)(clusteringPlan.get().getInputGroups.size()) - // No pending clustering instant - checkAnswer(s"call show_clustering(table => '$tableName')")() + // All clustering instants are completed + checkAnswer(s"call show_clustering(table => '$tableName', show_involved_partition => true)")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1000,ts=1001"), + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1002,ts=1003,ts=1004,ts=1005"), + Seq(resultC(0).head, resultC(0)(1), HoodieInstant.State.COMPLETED.name(), "ts=1006,ts=1007,ts=1009") + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 10.0, 1000), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala index f6e6772d161b6..39332d859171d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.hudi.procedure -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.hudi.common.table.timeline.HoodieInstant +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase -class TestCompactionProcedure extends TestHoodieSqlBase { +class TestCompactionProcedure extends HoodieSparkSqlTestBase { test("Test Call run_compaction Procedure by Table") { withTempDir { tmp => @@ -48,22 +49,52 @@ class TestCompactionProcedure extends TestHoodieSqlBase { spark.sql(s"insert into $tableName values(4, 'a4', 10, 1000)") spark.sql(s"update $tableName set price = 11 where id = 1") - spark.sql(s"call run_compaction(op => 'schedule', table => '$tableName')") + // Schedule the first compaction + val resultA = spark.sql(s"call run_compaction(op => 'schedule', table => '$tableName')") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2))) + spark.sql(s"update $tableName set price = 12 where id = 2") - spark.sql(s"call run_compaction('schedule', '$tableName')") - val compactionRows = spark.sql(s"call show_compaction(table => '$tableName', limit => 10)").collect() - val timestamps = compactionRows.map(_.getString(0)) + + // Schedule the second compaction + val resultB = spark.sql(s"call run_compaction('schedule', '$tableName')") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2))) + + assertResult(1)(resultA.length) + assertResult(1)(resultB.length) + val showCompactionSql: String = s"call show_compaction(table => '$tableName', limit => 10)" + checkAnswer(showCompactionSql)( + resultA(0), + resultB(0) + ) + + val compactionRows = spark.sql(showCompactionSql).collect() + val timestamps = compactionRows.map(_.getString(0)).sorted assertResult(2)(timestamps.length) - spark.sql(s"call run_compaction(op => 'run', table => '$tableName', timestamp => ${timestamps(1)})") + // Execute the second scheduled compaction instant actually + checkAnswer(s"call run_compaction(op => 'run', table => '$tableName', timestamp => ${timestamps(1)})")( + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name()) + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 11.0, 1000), Seq(2, "a2", 12.0, 1000), Seq(3, "a3", 10.0, 1000), Seq(4, "a4", 10.0, 1000) ) - assertResult(1)(spark.sql(s"call show_compaction('$tableName')").collect().length) - spark.sql(s"call run_compaction(op => 'run', table => '$tableName', timestamp => ${timestamps(0)})") + + // A compaction action eventually becomes commit when completed, so show_compaction + // can only see the first scheduled compaction instant + val resultC = spark.sql(s"call show_compaction('$tableName')") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2))) + assertResult(1)(resultC.length) + assertResult(resultA)(resultC) + + checkAnswer(s"call run_compaction(op => 'run', table => '$tableName', timestamp => ${timestamps(0)})")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name()) + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 11.0, 1000), Seq(2, "a2", 12.0, 1000), @@ -98,25 +129,40 @@ class TestCompactionProcedure extends TestHoodieSqlBase { spark.sql(s"insert into $tableName values(3, 'a3', 10, 1000)") spark.sql(s"update $tableName set price = 11 where id = 1") - spark.sql(s"call run_compaction(op => 'run', path => '${tmp.getCanonicalPath}')") + checkAnswer(s"call run_compaction(op => 'run', path => '${tmp.getCanonicalPath}')")() checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 11.0, 1000), Seq(2, "a2", 10.0, 1000), Seq(3, "a3", 10.0, 1000) ) assertResult(0)(spark.sql(s"call show_compaction(path => '${tmp.getCanonicalPath}')").collect().length) - // schedule compaction first + spark.sql(s"update $tableName set price = 12 where id = 1") - spark.sql(s"call run_compaction(op=> 'schedule', path => '${tmp.getCanonicalPath}')") - // schedule compaction second + // Schedule the first compaction + val resultA = spark.sql(s"call run_compaction(op=> 'schedule', path => '${tmp.getCanonicalPath}')") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2))) + spark.sql(s"update $tableName set price = 12 where id = 2") - spark.sql(s"call run_compaction(op => 'schedule', path => '${tmp.getCanonicalPath}')") - // show compaction - assertResult(2)(spark.sql(s"call show_compaction(path => '${tmp.getCanonicalPath}')").collect().length) - // run compaction for all the scheduled compaction - spark.sql(s"call run_compaction(op => 'run', path => '${tmp.getCanonicalPath}')") + // Schedule the second compaction + val resultB = spark.sql(s"call run_compaction(op => 'schedule', path => '${tmp.getCanonicalPath}')") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2))) + + assertResult(1)(resultA.length) + assertResult(1)(resultB.length) + checkAnswer(s"call show_compaction(path => '${tmp.getCanonicalPath}')")( + resultA(0), + resultB(0) + ) + + // Run compaction for all the scheduled compaction + checkAnswer(s"call run_compaction(op => 'run', path => '${tmp.getCanonicalPath}')")( + Seq(resultA(0).head, resultA(0)(1), HoodieInstant.State.COMPLETED.name()), + Seq(resultB(0).head, resultB(0)(1), HoodieInstant.State.COMPLETED.name()) + ) checkAnswer(s"select id, name, price, ts from $tableName order by id")( Seq(1, "a1", 12.0, 1000), diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala index 7d60ca018d32a..cfc5319c75641 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.hudi.procedure -import org.apache.spark.sql.hudi.TestHoodieSqlBase +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase -class TestSavepointsProcedure extends TestHoodieSqlBase { +class TestSavepointsProcedure extends HoodieSparkSqlTestBase { test("Test Call create_savepoints Procedure") { withTempDir { tmp => diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_2AvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_2AvroDeserializer.scala index 0275e2f635d3b..d839c73032cd4 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_2AvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3_2AvroDeserializer.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql.avro import org.apache.avro.Schema -import org.apache.hudi.HoodieSparkUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataType class HoodieSpark3_2AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) extends HoodieAvroDeserializer { - private val avroDeserializer = new AvroDeserializer(rootAvroType, rootCatalystType, "EXCEPTION") + private val avroDeserializer = new AvroDeserializer(rootAvroType, rootCatalystType, + SQLConf.get.getConf(SQLConf.AVRO_REBASE_MODE_IN_READ)) def deserialize(data: Any): Option[Any] = avroDeserializer.deserialize(data) } diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala index e20f934592e45..4c77733b144aa 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark3Analysis.scala @@ -179,21 +179,22 @@ case class HoodieSpark3ResolveReferences(sparkSession: SparkSession) extends Rul case class HoodieSpark3PostAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { plan match { - case ShowPartitions(child, specOpt, _) - if child.isInstanceOf[ResolvedTable] && - child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] => - ShowHoodieTablePartitionsCommand(child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier, specOpt.map(s => s.asInstanceOf[UnresolvedPartitionSpec].spec)) + case ShowPartitions(ResolvedTable(_, idt, _: HoodieInternalV2Table, _), specOpt, _) => + ShowHoodieTablePartitionsCommand( + idt.asTableIdentifier, specOpt.map(s => s.asInstanceOf[UnresolvedPartitionSpec].spec)) // Rewrite TruncateTableCommand to TruncateHoodieTableCommand - case TruncateTable(child) - if child.isInstanceOf[ResolvedTable] && - child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] => - new TruncateHoodieTableCommand(child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier, None) + case TruncateTable(ResolvedTable(_, idt, _: HoodieInternalV2Table, _)) => + TruncateHoodieTableCommand(idt.asTableIdentifier, None) - case DropPartitions(child, specs, ifExists, purge) - if child.resolved && child.isInstanceOf[ResolvedTable] && child.asInstanceOf[ResolvedTable].table.isInstanceOf[HoodieInternalV2Table] => + case TruncatePartition( + ResolvedTable(_, idt, _: HoodieInternalV2Table, _), + partitionSpec: UnresolvedPartitionSpec) => + TruncateHoodieTableCommand(idt.asTableIdentifier, Some(partitionSpec.spec)) + + case DropPartitions(ResolvedTable(_, idt, _: HoodieInternalV2Table, _), specs, ifExists, purge) => AlterHoodieTableDropPartitionCommand( - child.asInstanceOf[ResolvedTable].identifier.asTableIdentifier, + idt.asTableIdentifier, specs.seq.map(f => f.asInstanceOf[UnresolvedPartitionSpec]).map(s => s.spec), ifExists, purge, diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala index 82ea356215ca5..f30976f58ea26 100644 --- a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/hudi/catalog/HoodieCatalog.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.hudi.catalog import org.apache.hadoop.fs.Path import org.apache.hudi.exception.HoodieException -import org.apache.hudi.hive.util.ConfigUtils import org.apache.hudi.sql.InsertMode +import org.apache.hudi.sync.common.util.ConfigUtils import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport} import org.apache.spark.sql.HoodieSpark3SqlUtils.convertTransforms import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils, HoodieCatalogTable} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange, UpdateColumnComment, UpdateColumnType} @@ -89,19 +90,21 @@ class HoodieCatalog extends DelegatingCatalogExtension } override def loadTable(ident: Identifier): Table = { - try { - super.loadTable(ident) match { - case v1: V1Table if sparkAdapter.isHoodieTable(v1.catalogTable) => - HoodieInternalV2Table( - spark, - v1.catalogTable.location.toString, - catalogTable = Some(v1.catalogTable), - tableIdentifier = Some(ident.toString)) - case o => o - } - } catch { - case e: Exception => - throw e + super.loadTable(ident) match { + case V1Table(catalogTable0) if sparkAdapter.isHoodieTable(catalogTable0) => + val catalogTable = catalogTable0.comment match { + case Some(v) => + val newProps = catalogTable0.properties + (TableCatalog.PROP_COMMENT -> v) + catalogTable0.copy(properties = newProps) + case _ => + catalogTable0 + } + HoodieInternalV2Table( + spark = spark, + path = catalogTable.location.toString, + catalogTable = Some(catalogTable), + tableIdentifier = Some(ident.toString)) + case o => o } } @@ -213,7 +216,7 @@ class HoodieCatalog extends DelegatingCatalogExtension val loc = locUriOpt .orElse(existingTableOpt.flatMap(_.storage.locationUri)) .getOrElse(spark.sessionState.catalog.defaultTablePath(id)) - val storage = DataSource.buildStorageFormatFromOptions(writeOptions) + val storage = DataSource.buildStorageFormatFromOptions(writeOptions.--(needFilterProps)) .copy(locationUri = Option(loc)) val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED @@ -231,7 +234,7 @@ class HoodieCatalog extends DelegatingCatalogExtension provider = Option("hudi"), partitionColumnNames = newPartitionColumns, bucketSpec = newBucketSpec, - properties = tablePropertiesNew.asScala.toMap, + properties = tablePropertiesNew.asScala.toMap.--(needFilterProps), comment = commentOpt) val hoodieCatalogTable = HoodieCatalogTable(spark, tableDesc) diff --git a/hudi-sync/hudi-dla-sync/pom.xml b/hudi-sync/hudi-adb-sync/pom.xml similarity index 99% rename from hudi-sync/hudi-dla-sync/pom.xml rename to hudi-sync/hudi-adb-sync/pom.xml index 3770225ef7fcb..0dd8783b67133 100644 --- a/hudi-sync/hudi-dla-sync/pom.xml +++ b/hudi-sync/hudi-adb-sync/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - hudi-dla-sync + hudi-adb-sync jar diff --git a/hudi-sync/hudi-dla-sync/src/assembly/src.xml b/hudi-sync/hudi-adb-sync/src/assembly/src.xml similarity index 100% rename from hudi-sync/hudi-dla-sync/src/assembly/src.xml rename to hudi-sync/hudi-adb-sync/src/assembly/src.xml diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java new file mode 100644 index 0000000000000..84316ddb1152b --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.adb; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hive.PartitionValueExtractor; +import org.apache.hudi.hive.SchemaDifference; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public abstract class AbstractAdbSyncHoodieClient extends AbstractSyncHoodieClient { + protected AdbSyncConfig adbSyncConfig; + protected PartitionValueExtractor partitionValueExtractor; + protected HoodieTimeline activeTimeline; + + public AbstractAdbSyncHoodieClient(AdbSyncConfig syncConfig, FileSystem fs) { + super(syncConfig.basePath, syncConfig.assumeDatePartitioning, + syncConfig.useFileListingFromMetadata, false, fs); + this.adbSyncConfig = syncConfig; + final String clazz = adbSyncConfig.partitionValueExtractorClass; + try { + this.partitionValueExtractor = (PartitionValueExtractor) Class.forName(clazz).newInstance(); + } catch (Exception e) { + throw new HoodieException("Fail to init PartitionValueExtractor class " + clazz, e); + } + + activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + } + + public List getPartitionEvents(Map, String> tablePartitions, + List partitionStoragePartitions) { + Map paths = new HashMap<>(); + + for (Map.Entry, String> entry : tablePartitions.entrySet()) { + List partitionValues = entry.getKey(); + String fullTablePartitionPath = entry.getValue(); + paths.put(String.join(", ", partitionValues), fullTablePartitionPath); + } + List events = new ArrayList<>(); + for (String storagePartition : partitionStoragePartitions) { + Path storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, storagePartition); + String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + // Check if the partition values or if hdfs path is the same + List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); + if (adbSyncConfig.useHiveStylePartitioning) { + String partition = String.join("/", storagePartitionValues); + storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); + fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + } + if (!storagePartitionValues.isEmpty()) { + String storageValue = String.join(", ", storagePartitionValues); + if (!paths.containsKey(storageValue)) { + events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); + } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { + events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + } + } + } + return events; + } + + public void close() { + + } + + public abstract Map, String> scanTablePartitions(String tableName) throws Exception; + + public abstract void updateTableDefinition(String tableName, SchemaDifference schemaDiff) throws Exception; + + public abstract boolean databaseExists(String databaseName) throws Exception; + + public abstract void createDatabase(String databaseName) throws Exception; + + public abstract void dropTable(String tableName); + + protected String getDatabasePath() { + String dbLocation = adbSyncConfig.dbLocation; + Path dbLocationPath; + if (StringUtils.isNullOrEmpty(dbLocation)) { + if (new Path(adbSyncConfig.basePath).isRoot()) { + dbLocationPath = new Path(adbSyncConfig.basePath); + } else { + dbLocationPath = new Path(adbSyncConfig.basePath).getParent(); + } + } else { + dbLocationPath = new Path(dbLocation); + } + return generateAbsolutePathStr(dbLocationPath); + } + + protected String generateAbsolutePathStr(Path path) { + String absolutePathStr = path.toString(); + if (path.toUri().getScheme() == null) { + absolutePathStr = getDefaultFs() + absolutePathStr; + } + return absolutePathStr.endsWith("/") ? absolutePathStr : absolutePathStr + "/"; + } + + protected String getDefaultFs() { + return fs.getConf().get("fs.defaultFS"); + } +} diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java new file mode 100644 index 0000000000000..ae2e7024e5870 --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sync.adb; + +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; + +/** + * Configs needed to sync data into Alibaba Cloud AnalyticDB(ADB). + */ +public class AdbSyncConfig extends HoodieSyncConfig { + + @Parameter(names = {"--user"}, description = "Adb username", required = true) + public String adbUser; + + @Parameter(names = {"--pass"}, description = "Adb password", required = true) + public String adbPass; + + @Parameter(names = {"--jdbc-url"}, description = "Adb jdbc connect url", required = true) + public String jdbcUrl; + + @Parameter(names = {"--skip-ro-suffix"}, description = "Whether skip the `_ro` suffix for read optimized table when syncing") + public Boolean skipROSuffix; + + @Parameter(names = {"--skip-rt-sync"}, description = "Whether skip the rt table when syncing") + public Boolean skipRTSync; + + @Parameter(names = {"--hive-style-partitioning"}, description = "Whether use hive style partitioning, true if like the following style: field1=value1/field2=value2") + public Boolean useHiveStylePartitioning; + + @Parameter(names = {"--support-timestamp"}, description = "If true, converts int64(timestamp_micros) to timestamp type") + public Boolean supportTimestamp; + + @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table") + public Boolean syncAsSparkDataSourceTable; + + @Parameter(names = {"--table-properties"}, description = "Table properties, to support read hoodie table as datasource table", required = true) + public String tableProperties; + + @Parameter(names = {"--serde-properties"}, description = "Serde properties, to support read hoodie table as datasource table", required = true) + public String serdeProperties; + + @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; + + @Parameter(names = {"--db-location"}, description = "Database location") + public String dbLocation; + + @Parameter(names = {"--auto-create-database"}, description = "Whether auto create adb database") + public Boolean autoCreateDatabase = true; + + @Parameter(names = {"--skip-last-commit-time-sync"}, description = "Whether skip last commit time syncing") + public Boolean skipLastCommitTimeSync = false; + + @Parameter(names = {"--drop-table-before-creation"}, description = "Whether drop table before creation") + public Boolean dropTableBeforeCreation = false; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + public static final ConfigProperty ADB_SYNC_USER = ConfigProperty + .key("hoodie.datasource.adb.sync.username") + .noDefaultValue() + .withDocumentation("ADB username"); + + public static final ConfigProperty ADB_SYNC_PASS = ConfigProperty + .key("hoodie.datasource.adb.sync.password") + .noDefaultValue() + .withDocumentation("ADB user password"); + + public static final ConfigProperty ADB_SYNC_JDBC_URL = ConfigProperty + .key("hoodie.datasource.adb.sync.jdbc_url") + .noDefaultValue() + .withDocumentation("Adb jdbc connect url"); + + public static final ConfigProperty ADB_SYNC_SKIP_RO_SUFFIX = ConfigProperty + .key("hoodie.datasource.adb.sync.skip_ro_suffix") + .defaultValue(true) + .withDocumentation("Whether skip the `_ro` suffix for read optimized table when syncing"); + + public static final ConfigProperty ADB_SYNC_SKIP_RT_SYNC = ConfigProperty + .key("hoodie.datasource.adb.sync.skip_rt_sync") + .defaultValue(true) + .withDocumentation("Whether skip the rt table when syncing"); + + public static final ConfigProperty ADB_SYNC_USE_HIVE_STYLE_PARTITIONING = ConfigProperty + .key("hoodie.datasource.adb.sync.hive_style_partitioning") + .defaultValue(false) + .withDocumentation("Whether use hive style partitioning, true if like the following style: field1=value1/field2=value2"); + + public static final ConfigProperty ADB_SYNC_SUPPORT_TIMESTAMP = ConfigProperty + .key("hoodie.datasource.adb.sync.support_timestamp") + .defaultValue(false) + .withDocumentation("If true, converts int64(timestamp_micros) to timestamp type"); + + public static final ConfigProperty ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE = ConfigProperty + .key("hoodie.datasource.adb.sync.sync_as_spark_datasource") + .defaultValue(true) + .withDocumentation("Whether sync this table as spark data source table"); + + public static final ConfigProperty ADB_SYNC_TABLE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.adb.sync.table_properties") + .noDefaultValue() + .withDocumentation("Table properties, to support read hoodie table as datasource table"); + + public static final ConfigProperty ADB_SYNC_SERDE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.adb.sync.serde_properties") + .noDefaultValue() + .withDocumentation("Serde properties, to support read hoodie table as datasource table"); + + public static final ConfigProperty ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD = ConfigProperty + .key("hoodie.datasource.adb.sync.schema_string_length_threshold") + .defaultValue(4000) + .withDocumentation("The maximum length allowed in a single cell when storing additional schema information in Hive's metastore"); + + public static final ConfigProperty ADB_SYNC_DB_LOCATION = ConfigProperty + .key("hoodie.datasource.adb.sync.db_location") + .noDefaultValue() + .withDocumentation("Database location"); + + public static final ConfigProperty ADB_SYNC_AUTO_CREATE_DATABASE = ConfigProperty + .key("hoodie.datasource.adb.sync.auto_create_database") + .defaultValue(true) + .withDocumentation("Whether auto create adb database"); + + public static final ConfigProperty ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC = ConfigProperty + .key("hoodie.datasource.adb.sync.skip_last_commit_time_sync") + .defaultValue(false) + .withDocumentation("Whether skip last commit time syncing"); + + public static final ConfigProperty ADB_SYNC_DROP_TABLE_BEFORE_CREATION = ConfigProperty + .key("hoodie.datasource.adb.sync.drop_table_before_creation") + .defaultValue(false) + .withDocumentation("Whether drop table before creation"); + + public AdbSyncConfig() { + this(new TypedProperties()); + } + + public AdbSyncConfig(TypedProperties props) { + super(props); + + adbUser = getString(ADB_SYNC_USER); + adbPass = getString(ADB_SYNC_PASS); + jdbcUrl = getString(ADB_SYNC_JDBC_URL); + skipROSuffix = getBooleanOrDefault(ADB_SYNC_SKIP_RO_SUFFIX); + skipRTSync = getBooleanOrDefault(ADB_SYNC_SKIP_RT_SYNC); + useHiveStylePartitioning = getBooleanOrDefault(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING); + supportTimestamp = getBooleanOrDefault(ADB_SYNC_SUPPORT_TIMESTAMP); + syncAsSparkDataSourceTable = getBooleanOrDefault(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE); + tableProperties = getString(ADB_SYNC_TABLE_PROPERTIES); + serdeProperties = getString(ADB_SYNC_SERDE_PROPERTIES); + sparkSchemaLengthThreshold = getIntOrDefault(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD); + dbLocation = getString(ADB_SYNC_DB_LOCATION); + autoCreateDatabase = getBooleanOrDefault(ADB_SYNC_AUTO_CREATE_DATABASE); + skipLastCommitTimeSync = getBooleanOrDefault(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC); + dropTableBeforeCreation = getBooleanOrDefault(ADB_SYNC_DROP_TABLE_BEFORE_CREATION); + } + + public static TypedProperties toProps(AdbSyncConfig cfg) { + TypedProperties properties = new TypedProperties(); + properties.put(META_SYNC_DATABASE_NAME.key(), cfg.databaseName); + properties.put(META_SYNC_TABLE_NAME.key(), cfg.tableName); + properties.put(ADB_SYNC_USER.key(), cfg.adbUser); + properties.put(ADB_SYNC_PASS.key(), cfg.adbPass); + properties.put(ADB_SYNC_JDBC_URL.key(), cfg.jdbcUrl); + properties.put(META_SYNC_BASE_PATH.key(), cfg.basePath); + properties.put(META_SYNC_PARTITION_FIELDS.key(), String.join(",", cfg.partitionFields)); + properties.put(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), cfg.partitionValueExtractorClass); + properties.put(META_SYNC_ASSUME_DATE_PARTITION.key(), String.valueOf(cfg.assumeDatePartitioning)); + properties.put(ADB_SYNC_SKIP_RO_SUFFIX.key(), String.valueOf(cfg.skipROSuffix)); + properties.put(ADB_SYNC_SKIP_RT_SYNC.key(), String.valueOf(cfg.skipRTSync)); + properties.put(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING.key(), String.valueOf(cfg.useHiveStylePartitioning)); + properties.put(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(cfg.useFileListingFromMetadata)); + properties.put(ADB_SYNC_SUPPORT_TIMESTAMP.key(), String.valueOf(cfg.supportTimestamp)); + properties.put(ADB_SYNC_TABLE_PROPERTIES.key(), cfg.tableProperties); + properties.put(ADB_SYNC_SERDE_PROPERTIES.key(), cfg.serdeProperties); + properties.put(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE.key(), String.valueOf(cfg.syncAsSparkDataSourceTable)); + properties.put(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key(), String.valueOf(cfg.sparkSchemaLengthThreshold)); + properties.put(META_SYNC_SPARK_VERSION.key(), cfg.sparkVersion); + properties.put(ADB_SYNC_DB_LOCATION.key(), cfg.dbLocation); + properties.put(ADB_SYNC_AUTO_CREATE_DATABASE.key(), String.valueOf(cfg.autoCreateDatabase)); + properties.put(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC.key(), String.valueOf(cfg.skipLastCommitTimeSync)); + properties.put(ADB_SYNC_DROP_TABLE_BEFORE_CREATION.key(), String.valueOf(cfg.dropTableBeforeCreation)); + + return properties; + } + + @Override + public String toString() { + return "AdbSyncConfig{" + + "adbUser='" + adbUser + '\'' + + ", adbPass='" + adbPass + '\'' + + ", jdbcUrl='" + jdbcUrl + '\'' + + ", skipROSuffix=" + skipROSuffix + + ", skipRTSync=" + skipRTSync + + ", useHiveStylePartitioning=" + useHiveStylePartitioning + + ", supportTimestamp=" + supportTimestamp + + ", syncAsSparkDataSourceTable=" + syncAsSparkDataSourceTable + + ", tableProperties='" + tableProperties + '\'' + + ", serdeProperties='" + serdeProperties + '\'' + + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold + + ", dbLocation='" + dbLocation + '\'' + + ", autoCreateDatabase=" + autoCreateDatabase + + ", skipLastCommitTimeSync=" + skipLastCommitTimeSync + + ", dropTableBeforeCreation=" + dropTableBeforeCreation + + ", help=" + help + + ", databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", basePath='" + basePath + '\'' + + ", baseFileFormat='" + baseFileFormat + '\'' + + ", partitionFields=" + partitionFields + + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + + ", assumeDatePartitioning=" + assumeDatePartitioning + + ", decodePartition=" + decodePartition + + ", useFileListingFromMetadata=" + useFileListingFromMetadata + + ", isConditionalSync=" + isConditionalSync + + ", sparkVersion='" + sparkVersion + '\'' + + '}'; + } +} diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java new file mode 100644 index 0000000000000..8c2f9e20451ca --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.adb; + +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; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.hive.SchemaDifference; +import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; +import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.util.ConfigUtils; + +import com.beust.jcommander.JCommander; +import org.apache.hadoop.conf.Configuration; +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.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Adb sync tool is mainly used to sync hoodie tables to Alibaba Cloud AnalyticDB(ADB), + * it can be used as API `AdbSyncTool.syncHoodieTable(AdbSyncConfig)` or as command + * line `java -cp hoodie-hive.jar AdbSyncTool [args]` + * + *

+ * This utility will get the schema from the latest commit and will sync ADB table schema, + * incremental partitions will be synced as well. + */ +@SuppressWarnings("WeakerAccess") +public class AdbSyncTool extends AbstractSyncTool { + private static final Logger LOG = LoggerFactory.getLogger(AdbSyncTool.class); + + public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; + public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; + + private final AdbSyncConfig adbSyncConfig; + private final AbstractAdbSyncHoodieClient hoodieAdbClient; + private final String snapshotTableName; + private final Option roTableTableName; + + public AdbSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { + super(props, conf, fs); + this.adbSyncConfig = new AdbSyncConfig(props); + this.hoodieAdbClient = getHoodieAdbClient(adbSyncConfig, fs); + switch (hoodieAdbClient.getTableType()) { + case COPY_ON_WRITE: + this.snapshotTableName = adbSyncConfig.tableName; + this.roTableTableName = Option.empty(); + break; + case MERGE_ON_READ: + this.snapshotTableName = adbSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableTableName = adbSyncConfig.skipROSuffix ? Option.of(adbSyncConfig.tableName) + : Option.of(adbSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + break; + default: + throw new HoodieAdbSyncException("Unknown table type:" + hoodieAdbClient.getTableType() + + ", basePath:" + hoodieAdbClient.getBasePath()); + } + } + + private AbstractAdbSyncHoodieClient getHoodieAdbClient(AdbSyncConfig adbSyncConfig, FileSystem fs) { + return new HoodieAdbJdbcClient(adbSyncConfig, fs); + } + + @Override + public void syncHoodieTable() { + try { + switch (hoodieAdbClient.getTableType()) { + case COPY_ON_WRITE: + syncHoodieTable(snapshotTableName, false, false); + break; + case MERGE_ON_READ: + // Sync a ro table for MOR table + syncHoodieTable(roTableTableName.get(), false, true); + // Sync a rt table for MOR table + if (!adbSyncConfig.skipRTSync) { + syncHoodieTable(snapshotTableName, true, false); + } + break; + default: + throw new HoodieAdbSyncException("Unknown table type:" + hoodieAdbClient.getTableType() + + ", basePath:" + hoodieAdbClient.getBasePath()); + } + } catch (Exception re) { + throw new HoodieAdbSyncException("Sync hoodie table to ADB failed, tableName:" + adbSyncConfig.tableName, re); + } finally { + hoodieAdbClient.close(); + } + } + + private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, + boolean readAsOptimized) throws Exception { + LOG.info("Try to sync hoodie table, tableName:{}, path:{}, tableType:{}", + tableName, hoodieAdbClient.getBasePath(), hoodieAdbClient.getTableType()); + + if (adbSyncConfig.autoCreateDatabase) { + try { + synchronized (AdbSyncTool.class) { + if (!hoodieAdbClient.databaseExists(adbSyncConfig.databaseName)) { + hoodieAdbClient.createDatabase(adbSyncConfig.databaseName); + } + } + } catch (Exception e) { + throw new HoodieAdbSyncException("Failed to create database:" + adbSyncConfig.databaseName + + ", useRealtimeInputFormat = " + useRealtimeInputFormat, e); + } + } else if (!hoodieAdbClient.databaseExists(adbSyncConfig.databaseName)) { + throw new HoodieAdbSyncException("ADB database does not exists:" + adbSyncConfig.databaseName); + } + + // Currently HoodieBootstrapRelation does support reading bootstrap MOR rt table, + // so we disable the syncAsSparkDataSourceTable here to avoid read such kind table + // by the data source way (which will use the HoodieBootstrapRelation). + // TODO after we support bootstrap MOR rt table in HoodieBootstrapRelation[HUDI-2071], + // we can remove this logical. + if (hoodieAdbClient.isBootstrap() + && hoodieAdbClient.getTableType() == HoodieTableType.MERGE_ON_READ + && !readAsOptimized) { + adbSyncConfig.syncAsSparkDataSourceTable = false; + LOG.info("Disable sync as spark datasource table for mor rt table:{}", tableName); + } + + if (adbSyncConfig.dropTableBeforeCreation) { + LOG.info("Drop table before creation, tableName:{}", tableName); + hoodieAdbClient.dropTable(tableName); + } + + boolean tableExists = hoodieAdbClient.tableExists(tableName); + + // Get the parquet schema for this table looking at the latest commit + MessageType schema = hoodieAdbClient.getDataSchema(); + + // Sync schema if needed + syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema); + LOG.info("Sync schema complete, start syncing partitions for table:{}", tableName); + + // Get the last time we successfully synced partitions + Option lastCommitTimeSynced = Option.empty(); + if (tableExists) { + lastCommitTimeSynced = hoodieAdbClient.getLastCommitTimeSynced(tableName); + } + LOG.info("Last commit time synced was found:{}", lastCommitTimeSynced.orElse("null")); + + // Scan synced partitions + List writtenPartitionsSince; + if (adbSyncConfig.partitionFields.isEmpty()) { + writtenPartitionsSince = new ArrayList<>(); + } else { + writtenPartitionsSince = hoodieAdbClient.getPartitionsWrittenToSince(lastCommitTimeSynced); + } + LOG.info("Scan partitions complete, partitionNum:{}", writtenPartitionsSince.size()); + + // Sync the partitions if needed + syncPartitions(tableName, writtenPartitionsSince); + + // Update sync commit time + // whether to skip syncing commit time stored in tbl properties, since it is time consuming. + if (!adbSyncConfig.skipLastCommitTimeSync) { + hoodieAdbClient.updateLastCommitTimeSynced(tableName); + } + LOG.info("Sync complete for table:{}", tableName); + } + + /** + * Get the latest schema from the last commit and check if its in sync with the ADB + * table schema. If not, evolves the table schema. + * + * @param tableName The table to be synced + * @param tableExists Whether target table exists + * @param useRealTimeInputFormat Whether using realtime input format + * @param readAsOptimized Whether read as optimized table + * @param schema The extracted schema + */ + private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, + boolean readAsOptimized, MessageType schema) throws Exception { + // Append spark table properties & serde properties + Map tableProperties = ConfigUtils.toMap(adbSyncConfig.tableProperties); + Map serdeProperties = ConfigUtils.toMap(adbSyncConfig.serdeProperties); + if (adbSyncConfig.syncAsSparkDataSourceTable) { + Map sparkTableProperties = getSparkTableProperties(adbSyncConfig.partitionFields, + adbSyncConfig.sparkVersion, adbSyncConfig.sparkSchemaLengthThreshold, schema); + Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized, adbSyncConfig.basePath); + tableProperties.putAll(sparkTableProperties); + serdeProperties.putAll(sparkSerdeProperties); + LOG.info("Sync as spark datasource table, tableName:{}, tableExists:{}, tableProperties:{}, sederProperties:{}", + tableName, tableExists, tableProperties, serdeProperties); + } + + // Check and sync schema + if (!tableExists) { + LOG.info("ADB table [{}] is not found, creating it", tableName); + String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(HoodieFileFormat.PARQUET, useRealTimeInputFormat); + + // Custom serde will not work with ALTER TABLE REPLACE COLUMNS + // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive + // /ql/exec/DDLTask.java#L3488 + hoodieAdbClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), + ParquetHiveSerDe.class.getName(), serdeProperties, tableProperties); + } else { + // Check if the table schema has evolved + Map tableSchema = hoodieAdbClient.getTableSchema(tableName); + SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, adbSyncConfig.partitionFields, + adbSyncConfig.supportTimestamp); + if (!schemaDiff.isEmpty()) { + LOG.info("Schema difference found for table:{}", tableName); + hoodieAdbClient.updateTableDefinition(tableName, schemaDiff); + } else { + LOG.info("No Schema difference for table:{}", tableName); + } + } + } + + /** + * Syncs the list of storage partitions passed in (checks if the partition is in adb, if not adds it or if the + * partition path does not match, it updates the partition path). + */ + private void syncPartitions(String tableName, List writtenPartitionsSince) { + try { + if (adbSyncConfig.partitionFields.isEmpty()) { + LOG.info("Not a partitioned table."); + return; + } + + Map, String> partitions = hoodieAdbClient.scanTablePartitions(tableName); + List partitionEvents = hoodieAdbClient.getPartitionEvents(partitions, writtenPartitionsSince); + List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); + LOG.info("New Partitions:{}", newPartitions); + hoodieAdbClient.addPartitionsToTable(tableName, newPartitions); + List updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE); + LOG.info("Changed Partitions:{}", updatePartitions); + hoodieAdbClient.updatePartitionsToTable(tableName, updatePartitions); + } catch (Exception e) { + throw new HoodieAdbSyncException("Failed to sync partitions for table:" + tableName, e); + } + } + + private List filterPartitions(List events, PartitionEventType eventType) { + return events.stream().filter(s -> s.eventType == eventType) + .map(s -> s.storagePartition).collect(Collectors.toList()); + } + + public static void main(String[] args) { + // parse the params + final AdbSyncConfig cfg = new AdbSyncConfig(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + Configuration hadoopConf = new Configuration(); + FileSystem fs = FSUtils.getFs(cfg.basePath, hadoopConf); + new AdbSyncTool(AdbSyncConfig.toProps(cfg), hadoopConf, fs).syncHoodieTable(); + } +} diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java new file mode 100644 index 0000000000000..a347ba701110d --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java @@ -0,0 +1,440 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.adb; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.HoodieHiveSyncException; +import org.apache.hudi.hive.SchemaDifference; +import org.apache.hudi.hive.util.HiveSchemaUtil; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class HoodieAdbJdbcClient extends AbstractAdbSyncHoodieClient { + private static final Logger LOG = LoggerFactory.getLogger(HoodieAdbJdbcClient.class); + + public static final String HOODIE_LAST_COMMIT_TIME_SYNC = "hoodie_last_sync"; + // Make sure we have the jdbc driver in classpath + private static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; + public static final String ADB_ESCAPE_CHARACTER = ""; + private static final String TBL_PROPERTIES_STR = "TBLPROPERTIES"; + + static { + try { + Class.forName(DRIVER_NAME); + } catch (ClassNotFoundException e) { + throw new IllegalStateException("Could not find " + DRIVER_NAME + " in classpath. ", e); + } + } + + private Connection connection; + + public HoodieAdbJdbcClient(AdbSyncConfig syncConfig, FileSystem fs) { + super(syncConfig, fs); + createAdbConnection(); + LOG.info("Init adb jdbc client success, jdbcUrl:{}", syncConfig.jdbcUrl); + } + + private void createAdbConnection() { + if (connection == null) { + try { + Class.forName(DRIVER_NAME); + } catch (ClassNotFoundException e) { + LOG.error("Unable to load jdbc driver class", e); + return; + } + try { + this.connection = DriverManager.getConnection( + adbSyncConfig.jdbcUrl, adbSyncConfig.adbUser, adbSyncConfig.adbPass); + } catch (SQLException e) { + throw new HoodieException("Cannot create adb connection ", e); + } + } + } + + @Override + public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, + String outputFormatClass, String serdeClass, + Map serdeProperties, Map tableProperties) { + try { + LOG.info("Creating table:{}", tableName); + String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, + getHiveSyncConfig(), inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties); + executeAdbSql(createSQLQuery); + } catch (IOException e) { + throw new HoodieException("Fail to create table:" + tableName, e); + } + } + + @Override + public void dropTable(String tableName) { + LOG.info("Dropping table:{}", tableName); + String dropTable = "drop table if exists `" + adbSyncConfig.databaseName + "`.`" + tableName + "`"; + executeAdbSql(dropTable); + } + + public Map getTableSchema(String tableName) { + Map schema = new HashMap<>(); + ResultSet result = null; + try { + DatabaseMetaData databaseMetaData = connection.getMetaData(); + result = databaseMetaData.getColumns(adbSyncConfig.databaseName, + adbSyncConfig.databaseName, tableName, null); + while (result.next()) { + String columnName = result.getString(4); + String columnType = result.getString(6); + if ("DECIMAL".equals(columnType)) { + int columnSize = result.getInt("COLUMN_SIZE"); + int decimalDigits = result.getInt("DECIMAL_DIGITS"); + columnType += String.format("(%s,%s)", columnSize, decimalDigits); + } + schema.put(columnName, columnType); + } + return schema; + } catch (SQLException e) { + throw new HoodieException("Fail to get table schema:" + tableName, e); + } finally { + closeQuietly(result, null); + } + } + + @Override + public void addPartitionsToTable(String tableName, List partitionsToAdd) { + if (partitionsToAdd.isEmpty()) { + LOG.info("No partitions to add for table:{}", tableName); + return; + } + + LOG.info("Adding partitions to table:{}, partitionNum:{}", tableName, partitionsToAdd.size()); + String sql = constructAddPartitionsSql(tableName, partitionsToAdd); + executeAdbSql(sql); + } + + private void executeAdbSql(String sql) { + Statement stmt = null; + try { + stmt = connection.createStatement(); + LOG.info("Executing sql:{}", sql); + stmt.execute(sql); + } catch (SQLException e) { + throw new HoodieException("Fail to execute sql:" + sql, e); + } finally { + closeQuietly(null, stmt); + } + } + + private T executeQuerySQL(String sql, Function function) { + Statement stmt = null; + try { + stmt = connection.createStatement(); + LOG.info("Executing sql:{}", sql); + return function.apply(stmt.executeQuery(sql)); + } catch (SQLException e) { + throw new HoodieException("Fail to execute sql:" + sql, e); + } finally { + closeQuietly(null, stmt); + } + } + + public void createDatabase(String databaseName) { + String rootPath = getDatabasePath(); + LOG.info("Creating database:{}, databaseLocation:{}", databaseName, rootPath); + String sql = constructCreateDatabaseSql(rootPath); + executeAdbSql(sql); + } + + public boolean databaseExists(String databaseName) { + String sql = constructShowCreateDatabaseSql(databaseName); + Function transform = resultSet -> { + try { + return resultSet.next(); + } catch (Exception e) { + if (e.getMessage().contains("Unknown database `" + databaseName + "`")) { + return false; + } else { + throw new HoodieException("Fail to execute sql:" + sql, e); + } + } + }; + return executeQuerySQL(sql, transform); + } + + @Override + public boolean doesTableExist(String tableName) { + String sql = constructShowLikeTableSql(tableName); + Function transform = resultSet -> { + try { + return resultSet.next(); + } catch (Exception e) { + throw new HoodieException("Fail to execute sql:" + sql, e); + } + }; + return executeQuerySQL(sql, transform); + } + + @Override + public boolean tableExists(String tableName) { + return doesTableExist(tableName); + } + + @Override + public Option getLastCommitTimeSynced(String tableName) { + String sql = constructShowCreateTableSql(tableName); + + Function> transform = resultSet -> { + try { + if (resultSet.next()) { + String table = resultSet.getString(2); + Map attr = new HashMap<>(); + int index = table.indexOf(TBL_PROPERTIES_STR); + if (index != -1) { + String sub = table.substring(index + TBL_PROPERTIES_STR.length()); + sub = sub + .replaceAll("\\(", "") + .replaceAll("\\)", "") + .replaceAll("'", ""); + String[] str = sub.split(","); + + for (String s : str) { + String key = s.split("=")[0].trim(); + String value = s.split("=")[1].trim(); + attr.put(key, value); + } + } + return Option.ofNullable(attr.getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); + } + return Option.empty(); + } catch (Exception e) { + throw new HoodieException("Fail to execute sql:" + sql, e); + } + }; + return executeQuerySQL(sql, transform); + } + + @Override + public void updateLastCommitTimeSynced(String tableName) { + // Set the last commit time from the TBLProperties + String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); + try { + String sql = constructUpdateTblPropertiesSql(tableName, lastCommitSynced); + executeAdbSql(sql); + } catch (Exception e) { + throw new HoodieHiveSyncException("Fail to get update last commit time synced:" + lastCommitSynced, e); + } + } + + @Override + public Option getLastReplicatedTime(String tableName) { + throw new UnsupportedOperationException("Not support getLastReplicatedTime yet"); + } + + @Override + public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { + throw new UnsupportedOperationException("Not support updateLastReplicatedTimeStamp yet"); + } + + @Override + public void deleteLastReplicatedTimeStamp(String tableName) { + throw new UnsupportedOperationException("Not support deleteLastReplicatedTimeStamp yet"); + } + + @Override + public void updatePartitionsToTable(String tableName, List changedPartitions) { + if (changedPartitions.isEmpty()) { + LOG.info("No partitions to change for table:{}", tableName); + return; + } + + LOG.info("Changing partitions on table:{}, changedPartitionNum:{}", tableName, changedPartitions.size()); + List sqlList = constructChangePartitionsSql(tableName, changedPartitions); + for (String sql : sqlList) { + executeAdbSql(sql); + } + } + + @Override + public void dropPartitions(String tableName, List partitionsToDrop) { + throw new UnsupportedOperationException("Not support dropPartitions yet."); + } + + public Map, String> scanTablePartitions(String tableName) { + String sql = constructShowPartitionSql(tableName); + Function, String>> transform = resultSet -> { + Map, String> partitions = new HashMap<>(); + try { + while (resultSet.next()) { + if (resultSet.getMetaData().getColumnCount() > 0) { + String str = resultSet.getString(1); + if (!StringUtils.isNullOrEmpty(str)) { + List values = partitionValueExtractor.extractPartitionValuesInPath(str); + Path storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, String.join("/", values)); + String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + partitions.put(values, fullStoragePartitionPath); + } + } + } + } catch (Exception e) { + throw new HoodieException("Fail to execute sql:" + sql, e); + } + return partitions; + }; + return executeQuerySQL(sql, transform); + } + + public void updateTableDefinition(String tableName, SchemaDifference schemaDiff) { + LOG.info("Adding columns for table:{}", tableName); + schemaDiff.getAddColumnTypes().forEach((columnName, columnType) -> + executeAdbSql(constructAddColumnSql(tableName, columnName, columnType)) + ); + + LOG.info("Updating columns' definition for table:{}", tableName); + schemaDiff.getUpdateColumnTypes().forEach((columnName, columnType) -> + executeAdbSql(constructChangeColumnSql(tableName, columnName, columnType)) + ); + } + + private String constructAddPartitionsSql(String tableName, List partitions) { + StringBuilder sqlBuilder = new StringBuilder("alter table `"); + sqlBuilder.append(adbSyncConfig.databaseName).append("`").append(".`") + .append(tableName).append("`").append(" add if not exists "); + for (String partition : partitions) { + String partitionClause = getPartitionClause(partition); + Path partitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); + String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); + sqlBuilder.append(" partition (").append(partitionClause).append(") location '") + .append(fullPartitionPathStr).append("' "); + } + + return sqlBuilder.toString(); + } + + private List constructChangePartitionsSql(String tableName, List partitions) { + List changePartitions = new ArrayList<>(); + String useDatabase = "use `" + adbSyncConfig.databaseName + "`"; + changePartitions.add(useDatabase); + + String alterTable = "alter table `" + tableName + "`"; + for (String partition : partitions) { + String partitionClause = getPartitionClause(partition); + Path partitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); + String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); + String changePartition = alterTable + " add if not exists partition (" + partitionClause + + ") location '" + fullPartitionPathStr + "'"; + changePartitions.add(changePartition); + } + + return changePartitions; + } + + /** + * Generate Hive Partition from partition values. + * + * @param partition Partition path + * @return partition clause + */ + private String getPartitionClause(String partition) { + List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); + ValidationUtils.checkArgument(adbSyncConfig.partitionFields.size() == partitionValues.size(), + "Partition key parts " + adbSyncConfig.partitionFields + + " does not match with partition values " + partitionValues + ". Check partition strategy. "); + List partBuilder = new ArrayList<>(); + for (int i = 0; i < adbSyncConfig.partitionFields.size(); i++) { + partBuilder.add(adbSyncConfig.partitionFields.get(i) + "='" + partitionValues.get(i) + "'"); + } + + return String.join(",", partBuilder); + } + + private String constructShowPartitionSql(String tableName) { + return String.format("show partitions `%s`.`%s`", adbSyncConfig.databaseName, tableName); + } + + private String constructShowCreateTableSql(String tableName) { + return String.format("show create table `%s`.`%s`", adbSyncConfig.databaseName, tableName); + } + + private String constructShowLikeTableSql(String tableName) { + return String.format("show tables from `%s` like '%s'", adbSyncConfig.databaseName, tableName); + } + + private String constructCreateDatabaseSql(String rootPath) { + return String.format("create database if not exists `%s` with dbproperties(catalog = 'oss', location = '%s')", + adbSyncConfig.databaseName, rootPath); + } + + private String constructShowCreateDatabaseSql(String databaseName) { + return String.format("show create database `%s`", databaseName); + } + + private String constructUpdateTblPropertiesSql(String tableName, String lastCommitSynced) { + return String.format("alter table `%s`.`%s` set tblproperties('%s' = '%s')", + adbSyncConfig.databaseName, tableName, HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced); + } + + private String constructAddColumnSql(String tableName, String columnName, String columnType) { + return String.format("alter table `%s`.`%s` add columns(`%s` %s)", + adbSyncConfig.databaseName, tableName, columnName, columnType); + } + + private String constructChangeColumnSql(String tableName, String columnName, String columnType) { + return String.format("alter table `%s`.`%s` change `%s` `%s` %s", + adbSyncConfig.databaseName, tableName, columnName, columnName, columnType); + } + + private HiveSyncConfig getHiveSyncConfig() { + HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); + hiveSyncConfig.partitionFields = adbSyncConfig.partitionFields; + hiveSyncConfig.databaseName = adbSyncConfig.databaseName; + Path basePath = new Path(adbSyncConfig.basePath); + hiveSyncConfig.basePath = generateAbsolutePathStr(basePath); + return hiveSyncConfig; + } + + @Override + public void close() { + try { + if (connection != null) { + connection.close(); + } + } catch (SQLException e) { + LOG.error("Fail to close connection", e); + } + } +} diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbSyncException.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbSyncException.java new file mode 100644 index 0000000000000..0deb9b94cd525 --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbSyncException.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sync.adb; + +public class HoodieAdbSyncException extends RuntimeException { + public HoodieAdbSyncException(String message) { + super(message); + } + + public HoodieAdbSyncException(String message, Throwable t) { + super(message, t); + } +} diff --git a/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java b/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java new file mode 100644 index 0000000000000..f4eb8fc7fc453 --- /dev/null +++ b/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.sync.adb; + +import org.apache.hudi.common.config.TypedProperties; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +public class TestAdbSyncConfig { + @Test + public void testCopy() { + AdbSyncConfig adbSyncConfig = new AdbSyncConfig(); + adbSyncConfig.partitionFields = Arrays.asList("a", "b"); + adbSyncConfig.basePath = "/tmp"; + adbSyncConfig.assumeDatePartitioning = true; + adbSyncConfig.databaseName = "test"; + adbSyncConfig.tableName = "test"; + adbSyncConfig.adbUser = "adb"; + adbSyncConfig.adbPass = "adb"; + adbSyncConfig.jdbcUrl = "jdbc:mysql://localhost:3306"; + adbSyncConfig.skipROSuffix = false; + adbSyncConfig.tableProperties = "spark.sql.sources.provider= 'hudi'\\n" + + "spark.sql.sources.schema.numParts = '1'\\n " + + "spark.sql.sources.schema.part.0 ='xx'\\n " + + "spark.sql.sources.schema.numPartCols = '1'\\n" + + "spark.sql.sources.schema.partCol.0 = 'dt'"; + adbSyncConfig.serdeProperties = "'path'='/tmp/test_db/tbl'"; + adbSyncConfig.dbLocation = "file://tmp/test_db"; + + TypedProperties props = AdbSyncConfig.toProps(adbSyncConfig); + AdbSyncConfig copied = new AdbSyncConfig(props); + + assertEquals(copied.partitionFields, adbSyncConfig.partitionFields); + assertEquals(copied.basePath, adbSyncConfig.basePath); + assertEquals(copied.assumeDatePartitioning, adbSyncConfig.assumeDatePartitioning); + assertEquals(copied.databaseName, adbSyncConfig.databaseName); + assertEquals(copied.tableName, adbSyncConfig.tableName); + assertEquals(copied.adbUser, adbSyncConfig.adbUser); + assertEquals(copied.adbPass, adbSyncConfig.adbPass); + assertEquals(copied.basePath, adbSyncConfig.basePath); + assertEquals(copied.jdbcUrl, adbSyncConfig.jdbcUrl); + assertEquals(copied.skipROSuffix, adbSyncConfig.skipROSuffix); + assertEquals(copied.supportTimestamp, adbSyncConfig.supportTimestamp); + } +} diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties b/hudi-sync/hudi-adb-sync/src/test/resources/log4j-surefire-quiet.properties similarity index 100% rename from hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire-quiet.properties rename to hudi-sync/hudi-adb-sync/src/test/resources/log4j-surefire-quiet.properties diff --git a/hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties b/hudi-sync/hudi-adb-sync/src/test/resources/log4j-surefire.properties similarity index 100% rename from hudi-sync/hudi-dla-sync/src/test/resources/log4j-surefire.properties rename to hudi-sync/hudi-adb-sync/src/test/resources/log4j-surefire.properties diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java deleted file mode 100644 index d4d580fe276af..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncConfig.java +++ /dev/null @@ -1,111 +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.dla; - -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; - -import com.beust.jcommander.Parameter; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -/** - * Configs needed to sync data into DLA. - */ -public class DLASyncConfig implements Serializable { - - @Parameter(names = {"--database"}, description = "name of the target database in DLA", required = true) - public String databaseName; - - @Parameter(names = {"--table"}, description = "name of the target table in DLA", required = true) - public String tableName; - - @Parameter(names = {"--user"}, description = "DLA username", required = true) - public String dlaUser; - - @Parameter(names = {"--pass"}, description = "DLA password", required = true) - public String dlaPass; - - @Parameter(names = {"--jdbc-url"}, description = "DLA jdbc connect url", required = true) - public String jdbcUrl; - - @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 = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") - public Boolean skipROSuffix = false; - - @Parameter(names = {"--skip-rt-sync"}, description = "Skip the RT table syncing") - public Boolean skipRTSync = false; - - @Parameter(names = {"--hive-style-partitioning"}, description = "Use DLA hive style partitioning, true if like the following style: field1=value1/field2=value2") - public Boolean useDLASyncHiveStylePartitioning = false; - - @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") - public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; - - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; - - @Parameter(names = {"--support-timestamp"}, description = "If true, converts int64(timestamp_micros) to timestamp type") - public Boolean supportTimestamp = false; - - public static DLASyncConfig copy(DLASyncConfig cfg) { - DLASyncConfig newConfig = new DLASyncConfig(); - newConfig.databaseName = cfg.databaseName; - newConfig.tableName = cfg.tableName; - newConfig.dlaUser = cfg.dlaUser; - newConfig.dlaPass = cfg.dlaPass; - newConfig.jdbcUrl = cfg.jdbcUrl; - newConfig.basePath = cfg.basePath; - newConfig.partitionFields = cfg.partitionFields; - newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; - newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; - newConfig.skipROSuffix = cfg.skipROSuffix; - newConfig.skipRTSync = cfg.skipRTSync; - newConfig.useDLASyncHiveStylePartitioning = cfg.useDLASyncHiveStylePartitioning; - newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.supportTimestamp = cfg.supportTimestamp; - return newConfig; - } - - @Override - public String toString() { - return "DLASyncConfig{databaseName='" + databaseName + '\'' + ", tableName='" + tableName + '\'' - + ", dlaUser='" + dlaUser + '\'' + ", dlaPass='" + dlaPass + '\'' + ", jdbcUrl='" + jdbcUrl + '\'' - + ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='" - + partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning - + ", useDLASyncHiveStylePartitioning=" + useDLASyncHiveStylePartitioning - + ", useFileListingFromMetadata=" + useFileListingFromMetadata - + ", help=" + help + '}'; - } -} diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java deleted file mode 100644 index 97838d03ed66b..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java +++ /dev/null @@ -1,213 +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.dla; - -import com.beust.jcommander.JCommander; -import org.apache.hadoop.conf.Configuration; -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; -import org.apache.hudi.dla.util.Utils; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.InvalidTableException; -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.hudi.hive.SchemaDifference; -import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; -import org.apache.hudi.sync.common.AbstractSyncTool; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -/** - * Tool to sync a hoodie table with a dla table. Either use it as a api - * DLASyncTool.syncHoodieTable(DLASyncConfig) or as a command line java -cp hoodie-hive.jar DLASyncTool [args] - *

- * This utility will get the schema from the latest commit and will sync dla table schema Also this will sync the - * partitions incrementally (all the partitions modified since the last commit) - */ -@SuppressWarnings("WeakerAccess") -public class DLASyncTool extends AbstractSyncTool { - - private static final Logger LOG = LogManager.getLogger(DLASyncTool.class); - public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; - public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - - private final DLASyncConfig cfg; - private final HoodieDLAClient hoodieDLAClient; - private final String snapshotTableName; - private final Option roTableTableName; - - 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()) { - case COPY_ON_WRITE: - this.snapshotTableName = cfg.tableName; - this.roTableTableName = Option.empty(); - break; - case MERGE_ON_READ: - this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE; - this.roTableTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) : - Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE); - break; - default: - LOG.error("Unknown table type " + hoodieDLAClient.getTableType()); - throw new InvalidTableException(hoodieDLAClient.getBasePath()); - } - } - - @Override - public void syncHoodieTable() { - try { - switch (hoodieDLAClient.getTableType()) { - case COPY_ON_WRITE: - syncHoodieTable(snapshotTableName, false); - break; - case MERGE_ON_READ: - // sync a RO table for MOR - syncHoodieTable(roTableTableName.get(), false); - // sync a RT table for MOR - if (!cfg.skipRTSync) { - syncHoodieTable(snapshotTableName, true); - } - break; - default: - LOG.error("Unknown table type " + hoodieDLAClient.getTableType()); - throw new InvalidTableException(hoodieDLAClient.getBasePath()); - } - } catch (RuntimeException re) { - throw new HoodieException("Got runtime exception when dla syncing " + cfg.tableName, re); - } finally { - hoodieDLAClient.close(); - } - } - - private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat) { - LOG.info("Trying to sync hoodie table " + tableName + " with base path " + hoodieDLAClient.getBasePath() - + " of type " + hoodieDLAClient.getTableType()); - // Check if the necessary table exists - boolean tableExists = hoodieDLAClient.tableExists(tableName); - // Get the parquet schema for this table looking at the latest commit - MessageType schema = hoodieDLAClient.getDataSchema(); - // Sync schema if needed - syncSchema(tableName, tableExists, useRealtimeInputFormat, schema); - - LOG.info("Schema sync complete. Syncing partitions for " + tableName); - // Get the last time we successfully synced partitions - // TODO : once DLA supports alter table properties - Option lastCommitTimeSynced = Option.empty(); - /*if (tableExists) { - lastCommitTimeSynced = hoodieDLAClient.getLastCommitTimeSynced(tableName); - }*/ - LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null")); - List writtenPartitionsSince = hoodieDLAClient.getPartitionsWrittenToSince(lastCommitTimeSynced); - LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size()); - // Sync the partitions if needed - syncPartitions(tableName, writtenPartitionsSince); - - hoodieDLAClient.updateLastCommitTimeSynced(tableName); - LOG.info("Sync complete for " + tableName); - } - - /** - * Get the latest schema from the last commit and check if its in sync with the dla table schema. If not, evolves the - * table schema. - * - * @param tableExists - does table exist - * @param schema - extracted schema - */ - private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, MessageType schema) { - // Check and sync schema - if (!tableExists) { - LOG.info("DLA table " + tableName + " is not found. Creating it"); - - String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(HoodieFileFormat.PARQUET, useRealTimeInputFormat); - - // Custom serde will not work with ALTER TABLE REPLACE COLUMNS - // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive - // /ql/exec/DDLTask.java#L3488 - hoodieDLAClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), - ParquetHiveSerDe.class.getName(), new HashMap<>(), new HashMap<>()); - } else { - // Check if the table schema has evolved - Map tableSchema = hoodieDLAClient.getTableSchema(tableName); - SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields, cfg.supportTimestamp); - if (!schemaDiff.isEmpty()) { - LOG.info("Schema difference found for " + tableName); - hoodieDLAClient.updateTableDefinition(tableName, schemaDiff); - } else { - LOG.info("No Schema difference for " + tableName); - } - } - } - - /** - * Syncs the list of storage partitions passed in (checks if the partition is in dla, if not adds it or if the - * partition path does not match, it updates the partition path). - */ - private void syncPartitions(String tableName, List writtenPartitionsSince) { - try { - if (cfg.partitionFields.isEmpty()) { - LOG.info("not a partitioned table."); - return; - } - Map, String> partitions = hoodieDLAClient.scanTablePartitions(tableName); - List partitionEvents = - hoodieDLAClient.getPartitionEvents(partitions, writtenPartitionsSince); - List newPartitions = filterPartitions(partitionEvents, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType.ADD); - LOG.info("New Partitions " + newPartitions); - hoodieDLAClient.addPartitionsToTable(tableName, newPartitions); - List updatePartitions = filterPartitions(partitionEvents, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType.UPDATE); - LOG.info("Changed Partitions " + updatePartitions); - hoodieDLAClient.updatePartitionsToTable(tableName, updatePartitions); - } catch (Exception e) { - throw new HoodieException("Failed to sync partitions for table " + tableName, e); - } - } - - private List filterPartitions(List events, AbstractSyncHoodieClient.PartitionEvent.PartitionEventType eventType) { - return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition) - .collect(Collectors.toList()); - } - - public static void main(String[] args) { - // parse the params - final DLASyncConfig cfg = new DLASyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { - cmd.usage(); - System.exit(1); - } - 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/HoodieDLAClient.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java deleted file mode 100644 index 10869eaf27b64..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/HoodieDLAClient.java +++ /dev/null @@ -1,428 +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.dla; - -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.HoodieHiveSyncException; -import org.apache.hudi.hive.PartitionValueExtractor; -import org.apache.hudi.hive.SchemaDifference; -import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class HoodieDLAClient extends AbstractSyncHoodieClient { - private static final Logger LOG = LogManager.getLogger(HoodieDLAClient.class); - private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "hoodie_last_sync"; - // Make sure we have the dla JDBC driver in classpath - private static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; - private static final String DLA_ESCAPE_CHARACTER = ""; - private static final String TBL_PROPERTIES_STR = "TBLPROPERTIES"; - - static { - try { - Class.forName(DRIVER_NAME); - } catch (ClassNotFoundException e) { - throw new IllegalStateException("Could not find " + DRIVER_NAME + " in classpath. ", e); - } - } - - private Connection connection; - private DLASyncConfig dlaConfig; - private PartitionValueExtractor partitionValueExtractor; - - public HoodieDLAClient(DLASyncConfig syncConfig, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, - false, fs); - this.dlaConfig = syncConfig; - try { - this.partitionValueExtractor = - (PartitionValueExtractor) Class.forName(dlaConfig.partitionValueExtractorClass).newInstance(); - } catch (Exception e) { - throw new HoodieException( - "Failed to initialize PartitionValueExtractor class " + dlaConfig.partitionValueExtractorClass, e); - } - createDLAConnection(); - } - - private void createDLAConnection() { - if (connection == null) { - try { - Class.forName(DRIVER_NAME); - } catch (ClassNotFoundException e) { - LOG.error("Unable to load DLA driver class", e); - return; - } - try { - this.connection = DriverManager.getConnection(dlaConfig.jdbcUrl, dlaConfig.dlaUser, dlaConfig.dlaPass); - LOG.info("Successfully established DLA connection to " + dlaConfig.jdbcUrl); - } catch (SQLException e) { - throw new HoodieException("Cannot create dla connection ", e); - } - } - } - - @Override - public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, - String outputFormatClass, String serdeClass, - Map serdeProperties, Map tableProperties) { - try { - String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, toHiveSyncConfig(), - inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties); - LOG.info("Creating table with " + createSQLQuery); - updateDLASQL(createSQLQuery); - } catch (IOException e) { - throw new HoodieException("Failed to create table " + tableName, e); - } - } - - public Map getTableSchema(String tableName) { - if (!tableExists(tableName)) { - throw new IllegalArgumentException( - "Failed to get schema for table " + tableName + " does not exist"); - } - Map schema = new HashMap<>(); - ResultSet result = null; - try { - DatabaseMetaData databaseMetaData = connection.getMetaData(); - result = databaseMetaData.getColumns(dlaConfig.databaseName, dlaConfig.databaseName, tableName, null); - while (result.next()) { - TYPE_CONVERTOR.doConvert(result, schema); - } - return schema; - } catch (SQLException e) { - throw new HoodieException("Failed to get table schema for " + tableName, e); - } finally { - closeQuietly(result, null); - } - } - - @Override - public void addPartitionsToTable(String tableName, List partitionsToAdd) { - if (partitionsToAdd.isEmpty()) { - LOG.info("No partitions to add for " + tableName); - return; - } - LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + tableName); - String sql = constructAddPartitions(tableName, partitionsToAdd); - updateDLASQL(sql); - } - - public String constructAddPartitions(String tableName, List partitions) { - return constructDLAAddPartitions(tableName, partitions); - } - - String generateAbsolutePathStr(Path path) { - String absolutePathStr = path.toString(); - if (path.toUri().getScheme() == null) { - absolutePathStr = getDefaultFs() + absolutePathStr; - } - return absolutePathStr.endsWith("/") ? absolutePathStr : absolutePathStr + "/"; - } - - public List constructChangePartitions(String tableName, List partitions) { - List changePartitions = new ArrayList<>(); - String useDatabase = "USE " + DLA_ESCAPE_CHARACTER + dlaConfig.databaseName + DLA_ESCAPE_CHARACTER; - changePartitions.add(useDatabase); - String alterTable = "ALTER TABLE " + DLA_ESCAPE_CHARACTER + tableName + DLA_ESCAPE_CHARACTER; - for (String partition : partitions) { - String partitionClause = getPartitionClause(partition); - Path partitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition); - String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); - String changePartition = - alterTable + " ADD IF NOT EXISTS PARTITION (" + partitionClause + ") LOCATION '" + fullPartitionPathStr + "'"; - changePartitions.add(changePartition); - } - return changePartitions; - } - - /** - * Generate Hive Partition from partition values. - * - * @param partition Partition path - * @return - */ - public String getPartitionClause(String partition) { - List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); - ValidationUtils.checkArgument(dlaConfig.partitionFields.size() == partitionValues.size(), - "Partition key parts " + dlaConfig.partitionFields + " does not match with partition values " + partitionValues - + ". Check partition strategy. "); - List partBuilder = new ArrayList<>(); - for (int i = 0; i < dlaConfig.partitionFields.size(); i++) { - partBuilder.add(dlaConfig.partitionFields.get(i) + "='" + partitionValues.get(i) + "'"); - } - return partBuilder.stream().collect(Collectors.joining(",")); - } - - private String constructDLAAddPartitions(String tableName, List partitions) { - StringBuilder alterSQL = new StringBuilder("ALTER TABLE "); - alterSQL.append(DLA_ESCAPE_CHARACTER).append(dlaConfig.databaseName) - .append(DLA_ESCAPE_CHARACTER).append(".").append(DLA_ESCAPE_CHARACTER) - .append(tableName).append(DLA_ESCAPE_CHARACTER).append(" ADD IF NOT EXISTS "); - for (String partition : partitions) { - String partitionClause = getPartitionClause(partition); - Path partitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition); - String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); - alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '").append(fullPartitionPathStr) - .append("' "); - } - return alterSQL.toString(); - } - - private void updateDLASQL(String sql) { - Statement stmt = null; - try { - stmt = connection.createStatement(); - LOG.info("Executing SQL " + sql); - stmt.execute(sql); - } catch (SQLException e) { - throw new HoodieException("Failed in executing SQL " + sql, e); - } finally { - closeQuietly(null, stmt); - } - } - - @Override - public boolean doesTableExist(String tableName) { - return tableExists(tableName); - } - - @Override - public boolean tableExists(String tableName) { - String sql = consutructShowCreateTableSQL(tableName); - Statement stmt = null; - ResultSet rs = null; - try { - stmt = connection.createStatement(); - rs = stmt.executeQuery(sql); - } catch (SQLException e) { - return false; - } finally { - closeQuietly(rs, stmt); - } - return true; - } - - @Override - public Option getLastCommitTimeSynced(String tableName) { - String sql = consutructShowCreateTableSQL(tableName); - Statement stmt = null; - ResultSet rs = null; - try { - stmt = connection.createStatement(); - rs = stmt.executeQuery(sql); - if (rs.next()) { - String table = rs.getString(2); - Map attr = new HashMap<>(); - int index = table.indexOf(TBL_PROPERTIES_STR); - if (index != -1) { - String sub = table.substring(index + TBL_PROPERTIES_STR.length()); - sub = sub.replaceAll("\\(", "").replaceAll("\\)", "").replaceAll("'", ""); - String[] str = sub.split(","); - - for (int i = 0; i < str.length; i++) { - String key = str[i].split("=")[0].trim(); - String value = str[i].split("=")[1].trim(); - attr.put(key, value); - } - } - return Option.ofNullable(attr.getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); - } - return Option.empty(); - } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to get the last commit time synced from the table", e); - } finally { - closeQuietly(rs, stmt); - } - } - - @Override - public void updateLastCommitTimeSynced(String tableName) { - // TODO : dla do not support update tblproperties, so do nothing. - } - - @Override - public Option getLastReplicatedTime(String tableName) { - // no op; unsupported - return Option.empty(); - } - - @Override - public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - // no op; unsupported - } - - @Override - public void deleteLastReplicatedTimeStamp(String tableName) { - // no op; unsupported - } - - @Override - public void updatePartitionsToTable(String tableName, List changedPartitions) { - if (changedPartitions.isEmpty()) { - LOG.info("No partitions to change for " + tableName); - return; - } - LOG.info("Changing partitions " + changedPartitions.size() + " on " + tableName); - List sqls = constructChangePartitions(tableName, changedPartitions); - for (String sql : sqls) { - updateDLASQL(sql); - } - } - - @Override - public void dropPartitions(String tableName, List partitionsToDrop) { - throw new UnsupportedOperationException("Not support dropPartitions yet."); - } - - public Map, String> scanTablePartitions(String tableName) { - String sql = constructShowPartitionSQL(tableName); - Statement stmt = null; - ResultSet rs = null; - Map, String> partitions = new HashMap<>(); - try { - stmt = connection.createStatement(); - LOG.info("Executing SQL " + sql); - rs = stmt.executeQuery(sql); - while (rs.next()) { - if (rs.getMetaData().getColumnCount() > 0) { - String str = rs.getString(1); - if (!StringUtils.isNullOrEmpty(str)) { - List values = partitionValueExtractor.extractPartitionValuesInPath(str); - Path storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, String.join("/", values)); - String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - partitions.put(values, fullStoragePartitionPath); - } - } - } - return partitions; - } catch (SQLException e) { - throw new HoodieException("Failed in executing SQL " + sql, e); - } finally { - closeQuietly(rs, stmt); - } - } - - public List getPartitionEvents(Map, String> tablePartitions, List partitionStoragePartitions) { - Map paths = new HashMap<>(); - - for (Map.Entry, String> entry : tablePartitions.entrySet()) { - List partitionValues = entry.getKey(); - Collections.sort(partitionValues); - String fullTablePartitionPath = entry.getValue(); - paths.put(String.join(", ", partitionValues), fullTablePartitionPath); - } - List events = new ArrayList<>(); - for (String storagePartition : partitionStoragePartitions) { - Path storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, storagePartition); - String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - // Check if the partition values or if hdfs path is the same - List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); - if (dlaConfig.useDLASyncHiveStylePartitioning) { - String partition = String.join("/", storagePartitionValues); - storagePartitionPath = FSUtils.getPartitionPath(dlaConfig.basePath, partition); - fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - } - Collections.sort(storagePartitionValues); - if (!storagePartitionValues.isEmpty()) { - String storageValue = String.join(", ", storagePartitionValues); - if (!paths.containsKey(storageValue)) { - events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); - } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { - events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); - } - } - } - return events; - } - - public void updateTableDefinition(String tableName, SchemaDifference schemaDiff) { - ValidationUtils.checkArgument(schemaDiff.getDeleteColumns().size() == 0, "not support delete columns"); - ValidationUtils.checkArgument(schemaDiff.getUpdateColumnTypes().size() == 0, "not support alter column type"); - Map columns = schemaDiff.getAddColumnTypes(); - for (Map.Entry entry : columns.entrySet()) { - String columnName = entry.getKey(); - String columnType = entry.getValue(); - StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append(DLA_ESCAPE_CHARACTER) - .append(dlaConfig.databaseName).append(DLA_ESCAPE_CHARACTER).append(".") - .append(DLA_ESCAPE_CHARACTER).append(tableName) - .append(DLA_ESCAPE_CHARACTER).append(" ADD COLUMNS(") - .append(columnName).append(" ").append(columnType).append(" )"); - LOG.info("Updating table definition with " + sqlBuilder); - updateDLASQL(sqlBuilder.toString()); - } - } - - @Override - public void close() { - try { - if (connection != null) { - connection.close(); - } - } catch (SQLException e) { - LOG.error("Could not close connection ", e); - } - } - - private String constructShowPartitionSQL(String tableName) { - String sql = "show partitions " + dlaConfig.databaseName + "." + tableName; - return sql; - } - - private String consutructShowCreateTableSQL(String tableName) { - String sql = "show create table " + dlaConfig.databaseName + "." + tableName; - return sql; - } - - private String getDefaultFs() { - return fs.getConf().get("fs.defaultFS"); - } - - private HiveSyncConfig toHiveSyncConfig() { - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.partitionFields = dlaConfig.partitionFields; - hiveSyncConfig.databaseName = dlaConfig.databaseName; - Path basePath = new Path(dlaConfig.basePath); - hiveSyncConfig.basePath = generateAbsolutePathStr(basePath); - return hiveSyncConfig; - } -} 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 deleted file mode 100644 index d1b0dd4e9d56f..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java +++ /dev/null @@ -1,77 +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.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; - -public class Utils { - public static String DLA_DATABASE_OPT_KEY = "hoodie.datasource.dla_sync.database"; - public static String DLA_TABLE_OPT_KEY = "hoodie.datasource.dla_sync.table"; - public static String DLA_USER_OPT_KEY = "hoodie.datasource.dla_sync.username"; - public static String DLA_PASS_OPT_KEY = "hoodie.datasource.dla_sync.password"; - public static String DLA_URL_OPT_KEY = "hoodie.datasource.dla_sync.jdbcurl"; - public static String BATH_PATH = "basePath"; - public static String DLA_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.dla_sync.partition_fields"; - public static String DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.dla_sync.partition_extractor_class"; - public static String DLA_ASSUME_DATE_PARTITIONING = "hoodie.datasource.dla_sync.assume_date_partitioning"; - public static String DLA_SKIP_RO_SUFFIX = "hoodie.datasource.dla_sync.skip_ro_suffix"; - public static String DLA_SKIP_RT_SYNC = "hoodie.datasource.dla_sync.skip_rt_sync"; - public static String DLA_SYNC_HIVE_STYLE_PARTITIONING = "hoodie.datasource.dla_sync.hive.style.partitioning"; - - public static 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); - properties.put(DLA_PASS_OPT_KEY, cfg.dlaPass); - properties.put(DLA_URL_OPT_KEY, cfg.jdbcUrl); - properties.put(BATH_PATH, cfg.basePath); - properties.put(DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY, cfg.partitionValueExtractorClass); - properties.put(DLA_ASSUME_DATE_PARTITIONING, String.valueOf(cfg.assumeDatePartitioning)); - properties.put(DLA_SKIP_RO_SUFFIX, String.valueOf(cfg.skipROSuffix)); - properties.put(DLA_SYNC_HIVE_STYLE_PARTITIONING, String.valueOf(cfg.useDLASyncHiveStylePartitioning)); - return 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); - config.dlaUser = properties.getProperty(DLA_USER_OPT_KEY); - config.dlaPass = properties.getProperty(DLA_PASS_OPT_KEY); - config.jdbcUrl = properties.getProperty(DLA_URL_OPT_KEY); - config.basePath = properties.getProperty(BATH_PATH); - if (StringUtils.isNullOrEmpty(properties.getProperty(DLA_PARTITION_FIELDS_OPT_KEY))) { - config.partitionFields = new ArrayList<>(); - } else { - config.partitionFields = Arrays.asList(properties.getProperty(DLA_PARTITION_FIELDS_OPT_KEY).split(",")); - } - config.partitionValueExtractorClass = properties.getProperty(DLA_PARTITION_EXTRACTOR_CLASS_OPT_KEY); - config.assumeDatePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_ASSUME_DATE_PARTITIONING, "false")); - config.skipROSuffix = Boolean.parseBoolean(properties.getProperty(DLA_SKIP_RO_SUFFIX, "false")); - config.skipRTSync = Boolean.parseBoolean(properties.getProperty(DLA_SKIP_RT_SYNC, "false")); - config.useDLASyncHiveStylePartitioning = Boolean.parseBoolean(properties.getProperty(DLA_SYNC_HIVE_STYLE_PARTITIONING, "false")); - return config; - } -} diff --git a/hudi-sync/hudi-dla-sync/src/test/java/org/apache/hudi/dla/TestDLASyncConfig.java b/hudi-sync/hudi-dla-sync/src/test/java/org/apache/hudi/dla/TestDLASyncConfig.java deleted file mode 100644 index 366d5a24efb06..0000000000000 --- a/hudi-sync/hudi-dla-sync/src/test/java/org/apache/hudi/dla/TestDLASyncConfig.java +++ /dev/null @@ -1,55 +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.dla; - -import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertEquals; -public class TestDLASyncConfig { - @Test - public void testCopy() { - DLASyncConfig dlaSyncConfig = new DLASyncConfig(); - List partitions = Arrays.asList("a", "b"); - dlaSyncConfig.partitionFields = partitions; - dlaSyncConfig.basePath = "/tmp"; - dlaSyncConfig.assumeDatePartitioning = true; - dlaSyncConfig.databaseName = "test"; - dlaSyncConfig.tableName = "test"; - dlaSyncConfig.dlaUser = "dla"; - dlaSyncConfig.dlaPass = "dla"; - dlaSyncConfig.jdbcUrl = "jdbc:mysql://localhost:3306"; - dlaSyncConfig.skipROSuffix = false; - - DLASyncConfig copied = DLASyncConfig.copy(dlaSyncConfig); - - assertEquals(copied.partitionFields, dlaSyncConfig.partitionFields); - assertEquals(copied.basePath, dlaSyncConfig.basePath); - assertEquals(copied.assumeDatePartitioning, dlaSyncConfig.assumeDatePartitioning); - assertEquals(copied.databaseName, dlaSyncConfig.databaseName); - assertEquals(copied.tableName, dlaSyncConfig.tableName); - assertEquals(copied.dlaUser, dlaSyncConfig.dlaUser); - assertEquals(copied.dlaPass, dlaSyncConfig.dlaPass); - assertEquals(copied.basePath, dlaSyncConfig.basePath); - assertEquals(copied.jdbcUrl, dlaSyncConfig.jdbcUrl); - assertEquals(copied.skipROSuffix, dlaSyncConfig.skipROSuffix); - assertEquals(copied.supportTimestamp, dlaSyncConfig.supportTimestamp); - } -} 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 939fc114c0883..5e343b9a62a00 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 @@ -27,9 +27,8 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.InvalidTableException; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.hudi.hive.util.ConfigUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.hive.util.Parquet2SparkSchemaUtils; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.hudi.sync.common.AbstractSyncTool; @@ -43,20 +42,13 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import static org.apache.parquet.schema.OriginalType.UTF8; -import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; - /** * Tool to sync a hoodie HDFS table with a hive metastore table. Either use it as a api * HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive-sync.jar HiveSyncTool [args] @@ -248,8 +240,9 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea 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); + Map sparkTableProperties = getSparkTableProperties(hiveSyncConfig.partitionFields, + hiveSyncConfig.sparkVersion, hiveSyncConfig.sparkSchemaLengthThreshold, schema); + Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized, hiveSyncConfig.basePath); tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); } @@ -309,75 +302,6 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea return schemaChanged; } - /** - * Get Spark Sql related table properties. This is used for spark datasource table. - * @param schema The schema to write to the table. - * @return A new parameters added the spark's table properties. - */ - private Map getSparkTableProperties(int schemaLengthThreshold, MessageType schema) { - // Convert the schema and partition info used by spark sql to hive table properties. - // 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 = hiveSyncConfig.partitionFields; - List partitionCols = new ArrayList<>(); - List dataCols = new ArrayList<>(); - Map column2Field = new HashMap<>(); - - for (Type field : originGroupType.getFields()) { - column2Field.put(field.getName(), field); - } - // Get partition columns and data columns. - for (String partitionName : partitionNames) { - // Default the unknown partition fields to be String. - // Keep the same logical with HiveSchemaUtil#getPartitionKeyType. - partitionCols.add(column2Field.getOrDefault(partitionName, - new PrimitiveType(Type.Repetition.REQUIRED, BINARY, partitionName, UTF8))); - } - - for (Type field : originGroupType.getFields()) { - if (!partitionNames.contains(field.getName())) { - dataCols.add(field); - } - } - - List reOrderedFields = new ArrayList<>(); - reOrderedFields.addAll(dataCols); - reOrderedFields.addAll(partitionCols); - GroupType reOrderedType = new GroupType(originGroupType.getRepetition(), originGroupType.getName(), reOrderedFields); - - Map sparkProperties = new HashMap<>(); - sparkProperties.put("spark.sql.sources.provider", "hudi"); - 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); - int numSchemaPart = (schemaString.length() + schemaLengthThreshold - 1) / schemaLengthThreshold; - sparkProperties.put("spark.sql.sources.schema.numParts", String.valueOf(numSchemaPart)); - // Add each part of schema string to sparkProperties - for (int i = 0; i < numSchemaPart; i++) { - int start = i * schemaLengthThreshold; - int end = Math.min(start + schemaLengthThreshold, schemaString.length()); - sparkProperties.put("spark.sql.sources.schema.part." + i, schemaString.substring(start, end)); - } - // Add partition columns - if (!partitionNames.isEmpty()) { - sparkProperties.put("spark.sql.sources.schema.numPartCols", String.valueOf(partitionNames.size())); - for (int i = 0; i < partitionNames.size(); i++) { - sparkProperties.put("spark.sql.sources.schema.partCol." + i, partitionNames.get(i)); - } - } - return sparkProperties; - } - - private Map getSparkSerdeProperties(boolean readAsOptimized) { - Map sparkSerdeProperties = new HashMap<>(); - sparkSerdeProperties.put("path", hiveSyncConfig.basePath); - sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized)); - return sparkSerdeProperties; - } - /** * Syncs the list of storage partitions passed in (checks if the partition is in hive, if not adds it or if the * partition path does not match, it updates the partition path). 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 1c2d53ed96ded..b801f4d7daa11 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 @@ -28,7 +28,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.hive.testutils.HiveTestUtil; -import org.apache.hudi.hive.util.ConfigUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestParquet2SparkSchemaUtils.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestParquet2SparkSchemaUtils.java index 3ca31b04395a1..b6940629af3d2 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestParquet2SparkSchemaUtils.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestParquet2SparkSchemaUtils.java @@ -18,7 +18,7 @@ package org.apache.hudi.hive; -import org.apache.hudi.hive.util.Parquet2SparkSchemaUtils; +import org.apache.hudi.sync.common.util.Parquet2SparkSchemaUtils; import org.apache.spark.sql.execution.SparkSqlParser; import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter; import org.apache.spark.sql.internal.SQLConf; 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 680b4a17ef5d9..972ae1f96c512 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 @@ -18,12 +18,26 @@ package org.apache.hudi.sync.common; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.sync.common.util.Parquet2SparkSchemaUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Properties; +import static org.apache.parquet.schema.OriginalType.UTF8; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; + /** * Base class to sync Hudi meta data with Metastores to make * Hudi table queryable through external systems. @@ -46,4 +60,72 @@ public AbstractSyncTool(Properties props, FileSystem fileSystem) { public abstract void syncHoodieTable(); + /** + * Get Spark Sql related table properties. This is used for spark datasource table. + * @param schema The schema to write to the table. + * @return A new parameters added the spark's table properties. + */ + protected Map getSparkTableProperties(List partitionNames, String sparkVersion, + int schemaLengthThreshold, MessageType schema) { + // Convert the schema and partition info used by spark sql to hive table properties. + // 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 partitionCols = new ArrayList<>(); + List dataCols = new ArrayList<>(); + Map column2Field = new HashMap<>(); + + for (Type field : originGroupType.getFields()) { + column2Field.put(field.getName(), field); + } + // Get partition columns and data columns. + for (String partitionName : partitionNames) { + // Default the unknown partition fields to be String. + // Keep the same logical with HiveSchemaUtil#getPartitionKeyType. + partitionCols.add(column2Field.getOrDefault(partitionName, + new PrimitiveType(Type.Repetition.REQUIRED, BINARY, partitionName, UTF8))); + } + + for (Type field : originGroupType.getFields()) { + if (!partitionNames.contains(field.getName())) { + dataCols.add(field); + } + } + + List reOrderedFields = new ArrayList<>(); + reOrderedFields.addAll(dataCols); + reOrderedFields.addAll(partitionCols); + GroupType reOrderedType = new GroupType(originGroupType.getRepetition(), originGroupType.getName(), reOrderedFields); + + Map sparkProperties = new HashMap<>(); + sparkProperties.put("spark.sql.sources.provider", "hudi"); + if (!StringUtils.isNullOrEmpty(sparkVersion)) { + sparkProperties.put("spark.sql.create.version", sparkVersion); + } + // Split the schema string to multi-parts according the schemaLengthThreshold size. + String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType); + int numSchemaPart = (schemaString.length() + schemaLengthThreshold - 1) / schemaLengthThreshold; + sparkProperties.put("spark.sql.sources.schema.numParts", String.valueOf(numSchemaPart)); + // Add each part of schema string to sparkProperties + for (int i = 0; i < numSchemaPart; i++) { + int start = i * schemaLengthThreshold; + int end = Math.min(start + schemaLengthThreshold, schemaString.length()); + sparkProperties.put("spark.sql.sources.schema.part." + i, schemaString.substring(start, end)); + } + // Add partition columns + if (!partitionNames.isEmpty()) { + sparkProperties.put("spark.sql.sources.schema.numPartCols", String.valueOf(partitionNames.size())); + for (int i = 0; i < partitionNames.size(); i++) { + sparkProperties.put("spark.sql.sources.schema.partCol." + i, partitionNames.get(i)); + } + } + return sparkProperties; + } + + protected Map getSparkSerdeProperties(boolean readAsOptimized, String basePath) { + Map sparkSerdeProperties = new HashMap<>(); + sparkSerdeProperties.put("path", 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/util/ConfigUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java similarity index 98% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/ConfigUtils.java rename to hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java index 94ebdaadd8ff3..ca5224aef4697 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/ConfigUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.hive.util; +package org.apache.hudi.sync.common.util; import java.util.HashMap; import java.util.Map; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/Parquet2SparkSchemaUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java similarity index 99% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/Parquet2SparkSchemaUtils.java rename to hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java index debc262b5518c..c5b98c17eb4a1 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/Parquet2SparkSchemaUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/Parquet2SparkSchemaUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.hive.util; +package org.apache.hudi.sync.common.util; import org.apache.hudi.common.util.ValidationUtils; import org.apache.parquet.schema.GroupType; diff --git a/hudi-sync/pom.xml b/hudi-sync/pom.xml index 0ee145418f5ee..ffcbac8a652ef 100644 --- a/hudi-sync/pom.xml +++ b/hudi-sync/pom.xml @@ -32,7 +32,7 @@ hudi-datahub-sync - hudi-dla-sync + hudi-adb-sync hudi-hive-sync hudi-sync-common diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java index b5d7dc4b107dd..a2e81a3371d10 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java @@ -174,7 +174,7 @@ protected JavaRDD> buildHoodieRecordsForImport ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class)); HoodieEngineContext context = new HoodieSparkEngineContext(jsc); - context.setJobStatus(this.getClass().getSimpleName(), "Build records for import"); + context.setJobStatus(this.getClass().getSimpleName(), "Build records for import: " + cfg.tableName); return jsc.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration()) // To reduce large number of tasks. diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index a2717a35617f3..402b380a00e08 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -107,7 +107,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi fs.delete(new Path(outputDir), true); } - context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot"); + context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot: " + baseDir); List> filesToCopy = context.flatMap(partitions, partition -> { // Only take latest version files <= latestCommit. diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index 255393b232eb1..753765fb6a504 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -177,7 +177,7 @@ private void exportAsNonHudi(JavaSparkContext jsc, Config cfg, List part : ReflectionUtils.loadClass(cfg.outputPartitioner); HoodieEngineContext context = new HoodieSparkEngineContext(jsc); - context.setJobStatus(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset"); + context.setJobStatus(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset: " + cfg.targetOutputPath); final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg); Iterator exportingFilePaths = jsc .parallelize(partitions, partitions.size()) 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 b086a6c9edbab..a1a804b9ed123 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 @@ -37,6 +37,7 @@ import org.apache.hudi.common.model.WriteOperationType; 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; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -77,6 +78,7 @@ import com.codahale.metrics.Timer; import org.apache.avro.Schema; +import org.apache.avro.SchemaCompatibility; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -794,7 +796,7 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { .withProps(props); if (schema != null) { - builder.withSchema(schema.toString()); + builder.withSchema(getSchemaForWriteConfig(schema).toString()); } HoodieWriteConfig config = builder.build(); @@ -829,6 +831,25 @@ private HoodieWriteConfig getHoodieClientConfig(Schema schema) { return config; } + private Schema getSchemaForWriteConfig(Schema targetSchema) { + Schema newWriteSchema = targetSchema; + try { + if (targetSchema != null) { + // check if targetSchema is equal to NULL schema + if (SchemaCompatibility.checkReaderWriterCompatibility(targetSchema, InputBatch.NULL_SCHEMA).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE + && SchemaCompatibility.checkReaderWriterCompatibility(InputBatch.NULL_SCHEMA, targetSchema).getType() == SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE) { + // target schema is null. fetch schema from commit metadata and use it + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(new Configuration(fs.getConf())).setBasePath(cfg.targetBasePath).setPayloadClassName(cfg.payloadClassName).build(); + TableSchemaResolver schemaResolver = new TableSchemaResolver(meta); + newWriteSchema = schemaResolver.getTableAvroSchema(false); + } + } + return newWriteSchema; + } catch (Exception e) { + throw new HoodieException("Failed to fetch schema from table ", e); + } + } + /** * Register Avro Schemas. * diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 56124b82afc06..7a688b50c7097 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -43,6 +43,7 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieClusteringConfig; @@ -403,6 +404,9 @@ public static class Config implements Serializable { + "https://spark.apache.org/docs/latest/job-scheduling.html") public Integer clusterSchedulingMinShare = 0; + @Parameter(names = {"--post-write-termination-strategy-class"}, description = "Post writer termination strategy class to gracefully shutdown deltastreamer in continuous mode") + public String postWriteTerminationStrategyClass = ""; + public boolean isAsyncCompactionEnabled() { return continuousMode && !forceDisableCompaction && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType)); @@ -603,6 +607,8 @@ public static class DeltaSyncService extends HoodieAsyncService { */ private transient DeltaSync deltaSync; + private final Option postWriteTerminationStrategy; + public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf, Option properties) throws IOException { this.cfg = cfg; @@ -610,6 +616,8 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate(); this.asyncCompactService = Option.empty(); this.asyncClusteringService = Option.empty(); + this.postWriteTerminationStrategy = StringUtils.isNullOrEmpty(cfg.postWriteTerminationStrategyClass) ? Option.empty() : + TerminationStrategyUtils.createPostWriteTerminationStrategy(properties.get(), cfg.postWriteTerminationStrategyClass); if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = @@ -625,8 +633,8 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config ValidationUtils.checkArgument(baseFileFormat.equals(cfg.baseFileFormat) || cfg.baseFileFormat == null, "Hoodie table's base file format is of type " + baseFileFormat + " but passed in CLI argument is " + cfg.baseFileFormat); - cfg.baseFileFormat = meta.getTableConfig().getBaseFileFormat().toString(); - this.cfg.baseFileFormat = cfg.baseFileFormat; + cfg.baseFileFormat = baseFileFormat; + this.cfg.baseFileFormat = baseFileFormat; } else { tableType = HoodieTableType.valueOf(cfg.tableType); if (cfg.baseFileFormat == null) { @@ -695,6 +703,14 @@ protected Pair startService() { } } } + // check if deltastreamer need to be shutdown + if (postWriteTerminationStrategy.isPresent()) { + if (postWriteTerminationStrategy.get().shouldShutdown(scheduledCompactionInstantAndRDD.isPresent() ? Option.of(scheduledCompactionInstantAndRDD.get().getRight()) : + Option.empty())) { + error = true; + shutdown(false); + } + } long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start); if (toSleepMs > 0) { LOG.info("Last sync ran less than min sync interval: " + cfg.minSyncIntervalSeconds + " s, sleep: " diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/NoNewDataTerminationStrategy.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/NoNewDataTerminationStrategy.java new file mode 100644 index 0000000000000..2701ce4bc3085 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/NoNewDataTerminationStrategy.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.utilities.deltastreamer; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +/** + * Post writer termination strategy for deltastreamer in continuous mode. This strategy is based on no new data for consecutive number of times. + */ +public class NoNewDataTerminationStrategy implements PostWriteTerminationStrategy { + + private static final Logger LOG = LogManager.getLogger(NoNewDataTerminationStrategy.class); + + public static final String MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = "max.rounds.without.new.data.to.shutdown"; + public static final int DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN = 3; + + private final int numTimesNoNewDataToShutdown; + private int numTimesNoNewData = 0; + + public NoNewDataTerminationStrategy(TypedProperties properties) { + numTimesNoNewDataToShutdown = properties.getInteger(MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN, DEFAULT_MAX_ROUNDS_WITHOUT_NEW_DATA_TO_SHUTDOWN); + } + + @Override + public boolean shouldShutdown(Option> writeStatuses) { + numTimesNoNewData = writeStatuses.isPresent() ? 0 : numTimesNoNewData + 1; + if (numTimesNoNewData >= numTimesNoNewDataToShutdown) { + LOG.info("Shutting down on continuous mode as there is no new data for " + numTimesNoNewData); + return true; + } + return false; + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/PostWriteTerminationStrategy.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/PostWriteTerminationStrategy.java new file mode 100644 index 0000000000000..61f55428f166a --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/PostWriteTerminationStrategy.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.utilities.deltastreamer; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.util.Option; + +import org.apache.spark.api.java.JavaRDD; + +/** + * Post write termination strategy for deltastreamer in continuous mode. + */ +public interface PostWriteTerminationStrategy { + + /** + * Returns whether deltastreamer needs to be shutdown. + * @param writeStatuses optional pair of scheduled compaction instant and write statuses. + * @return true if deltastreamer has to be shutdown. false otherwise. + */ + boolean shouldShutdown(Option> writeStatuses); + +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/TerminationStrategyUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/TerminationStrategyUtils.java new file mode 100644 index 0000000000000..1b046a0db0da2 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/TerminationStrategyUtils.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.utilities.deltastreamer; + +import org.apache.hudi.common.config.TypedProperties; +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.exception.HoodieException; + +public class TerminationStrategyUtils { + + /** + * Create a PostWriteTerminationStrategy class via reflection, + *
+ * if the class name of PostWriteTerminationStrategy is configured through the {@link HoodieDeltaStreamer.Config#postWriteTerminationStrategyClass}. + */ + public static Option createPostWriteTerminationStrategy(TypedProperties properties, String postWriteTerminationStrategyClass) + throws HoodieException { + try { + return StringUtils.isNullOrEmpty(postWriteTerminationStrategyClass) + ? Option.empty() : + Option.of((PostWriteTerminationStrategy) ReflectionUtils.loadClass(postWriteTerminationStrategyClass, properties)); + } catch (Throwable e) { + throw new HoodieException("Could not create PostWritTerminationStrategy class " + postWriteTerminationStrategyClass, e); + } + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java index b2f6f784ca98b..04e3a574dc5c0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/InputBatch.java @@ -28,6 +28,7 @@ public class InputBatch { + public static final Schema NULL_SCHEMA = Schema.create(Schema.Type.NULL); private final Option batch; private final String checkpointForNextBatch; private final SchemaProvider schemaProvider; @@ -69,7 +70,7 @@ public NullSchemaProvider(TypedProperties props, JavaSparkContext jssc) { @Override public Schema getSourceSchema() { - return Schema.create(Schema.Type.NULL); + return NULL_SCHEMA; } } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java index 564c5e2058453..1abd2616b9be5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java @@ -22,17 +22,17 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; -import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; - import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException; import org.apache.hudi.utilities.sources.AvroKafkaSource; + import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; @@ -48,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -169,9 +170,14 @@ public static class Config { .withDocumentation("Kafka topic name."); public static final ConfigProperty KAFKA_CHECKPOINT_TYPE = ConfigProperty - .key("hoodie.deltastreamer.source.kafka.checkpoint.type") - .defaultValue("string") - .withDocumentation("Kafka chepoint type."); + .key("hoodie.deltastreamer.source.kafka.checkpoint.type") + .defaultValue("string") + .withDocumentation("Kafka checkpoint type."); + + public static final ConfigProperty KAFKA_FETCH_PARTITION_TIME_OUT = ConfigProperty + .key("hoodie.deltastreamer.source.kafka.fetch_partition.time.out") + .defaultValue(300 * 1000L) + .withDocumentation("Time out for fetching partitions. 5min by default"); public static final ConfigProperty ENABLE_KAFKA_COMMIT_OFFSET = ConfigProperty .key("hoodie.deltastreamer.source.kafka.enable.commit.offset") @@ -236,8 +242,7 @@ public OffsetRange[] getNextOffsetRanges(Option lastCheckpointStr, long if (!checkTopicExists(consumer)) { throw new HoodieException("Kafka topic:" + topicName + " does not exist"); } - List partitionInfoList; - partitionInfoList = consumer.partitionsFor(topicName); + List partitionInfoList = fetchPartitionInfos(consumer, topicName); Set topicPartitions = partitionInfoList.stream() .map(x -> new TopicPartition(x.topic(), x.partition())).collect(Collectors.toSet()); @@ -287,6 +292,32 @@ public OffsetRange[] getNextOffsetRanges(Option lastCheckpointStr, long return CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents); } + /** + * Fetch partition infos for given topic. + * + * @param consumer + * @param topicName + */ + private List fetchPartitionInfos(KafkaConsumer consumer, String topicName) { + long timeout = this.props.getLong(Config.KAFKA_FETCH_PARTITION_TIME_OUT.key(), Config.KAFKA_FETCH_PARTITION_TIME_OUT.defaultValue()); + long start = System.currentTimeMillis(); + + List partitionInfos; + do { + partitionInfos = consumer.partitionsFor(topicName); + try { + TimeUnit.SECONDS.sleep(10); + } catch (InterruptedException e) { + LOG.error("Sleep failed while fetching partitions"); + } + } while (partitionInfos == null && (System.currentTimeMillis() <= (start + timeout))); + + if (partitionInfos == null) { + throw new HoodieDeltaStreamerException(String.format("Can not find metadata for topic %s from kafka cluster", topicName)); + } + return partitionInfos; + } + /** * Fetch checkpoint offsets for each partition. * @param consumer instance of {@link KafkaConsumer} to fetch offsets from. 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 a9de85ce5ac9e..1a1cf39dbfef6 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 @@ -30,6 +30,7 @@ import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; +import org.apache.hudi.utilities.sources.TestDataSource; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.avro.Schema; @@ -191,6 +192,7 @@ protected static void writeCommonPropsToFile(FileSystem dfs, String dfsBasePath) @BeforeEach public void setup() throws Exception { super.setup(); + TestDataSource.returnEmptyBatch = false; } @AfterAll diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 0576f6aaee88b..ad94ada59b2fd 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -60,6 +60,7 @@ import org.apache.hudi.utilities.HoodieIndexer; import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; +import org.apache.hudi.utilities.deltastreamer.NoNewDataTerminationStrategy; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; @@ -738,18 +739,30 @@ public void testUpsertsCOWContinuousMode() throws Exception { testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow"); } + @Test + public void testUpsertsCOWContinuousModeShutdownGracefully() throws Exception { + testUpsertsContinuousMode(HoodieTableType.COPY_ON_WRITE, "continuous_cow", true); + } + @Test public void testUpsertsMORContinuousMode() throws Exception { testUpsertsContinuousMode(HoodieTableType.MERGE_ON_READ, "continuous_mor"); } private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir) throws Exception { + testUpsertsContinuousMode(tableType, tempDir, false); + } + + private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir, boolean testShutdownGracefully) throws Exception { String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); cfg.continuousMode = true; + if (testShutdownGracefully) { + cfg.postWriteTerminationStrategyClass = NoNewDataTerminationStrategy.class.getName(); + } cfg.tableType = tableType.name(); cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN.key())); @@ -763,6 +776,9 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir } TestHelpers.assertRecordCount(totalRecords, tableBasePath, sqlContext); TestHelpers.assertDistanceCount(totalRecords, tableBasePath, sqlContext); + if (testShutdownGracefully) { + TestDataSource.returnEmptyBatch = true; + } return true; }); } @@ -781,8 +797,35 @@ static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer. } }); TestHelpers.waitTillCondition(condition, dsFuture, 360); - ds.shutdownGracefully(); - dsFuture.get(); + if (cfg != null && !cfg.postWriteTerminationStrategyClass.isEmpty()) { + awaitDeltaStreamerShutdown(ds); + } else { + ds.shutdownGracefully(); + dsFuture.get(); + } + } + + static void awaitDeltaStreamerShutdown(HoodieDeltaStreamer ds) throws InterruptedException { + // await until deltastreamer shuts down on its own + boolean shutDownRequested = false; + int timeSoFar = 0; + while (!shutDownRequested) { + shutDownRequested = ds.getDeltaSyncService().isShutdownRequested(); + Thread.sleep(500); + timeSoFar += 500; + if (timeSoFar > (2 * 60 * 1000)) { + Assertions.fail("Deltastreamer should have shutdown by now"); + } + } + boolean shutdownComplete = false; + while (!shutdownComplete) { + shutdownComplete = ds.getDeltaSyncService().isShutdown(); + Thread.sleep(500); + timeSoFar += 500; + if (timeSoFar > (2 * 60 * 1000)) { + Assertions.fail("Deltastreamer should have shutdown by now"); + } + } } static void deltaStreamerTestRunner(HoodieDeltaStreamer ds, Function condition) throws Exception { @@ -1466,20 +1509,24 @@ private static void prepareJsonKafkaDFSFiles(int numRecords, boolean createTopic testUtils.sendMessages(topicName, Helpers.jsonifyRecords(dataGenerator.generateInsertsAsPerSchema("000", numRecords, HoodieTestDataGenerator.TRIP_SCHEMA))); } - private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer) throws IOException { + private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer, String emptyBatchParam) throws IOException { prepareParquetDFSSource(useSchemaProvider, hasTransformer, "source.avsc", "target.avsc", - PROPS_FILENAME_TEST_PARQUET, PARQUET_SOURCE_ROOT, false); + PROPS_FILENAME_TEST_PARQUET, PARQUET_SOURCE_ROOT, false, "partition_path", emptyBatchParam); + } + + private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer) throws IOException { + prepareParquetDFSSource(useSchemaProvider, hasTransformer, ""); } private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer, String sourceSchemaFile, String targetSchemaFile, - String propsFileName, String parquetSourceRoot, boolean addCommonProps) throws IOException { + String propsFileName, String parquetSourceRoot, boolean addCommonProps, String partitionPath) throws IOException { prepareParquetDFSSource(useSchemaProvider, hasTransformer, sourceSchemaFile, targetSchemaFile, propsFileName, parquetSourceRoot, addCommonProps, - "partition_path"); + partitionPath, ""); } private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTransformer, String sourceSchemaFile, String targetSchemaFile, String propsFileName, String parquetSourceRoot, boolean addCommonProps, - String partitionPath) throws IOException { + String partitionPath, String emptyBatchParam) throws IOException { // Properties used for testing delta-streamer with Parquet source TypedProperties parquetProps = new TypedProperties(); @@ -1498,6 +1545,9 @@ private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTrans } } parquetProps.setProperty("hoodie.deltastreamer.source.dfs.root", parquetSourceRoot); + if (!StringUtils.isNullOrEmpty(emptyBatchParam)) { + parquetProps.setProperty(TestParquetDFSSourceEmptyBatch.RETURN_EMPTY_BATCH, emptyBatchParam); + } UtilitiesTestBase.Helpers.savePropsToDFS(parquetProps, dfs, dfsBasePath + "/" + propsFileName); } @@ -1506,7 +1556,13 @@ private void testParquetDFSSource(boolean useSchemaProvider, List transf } private void testParquetDFSSource(boolean useSchemaProvider, List transformerClassNames, boolean testEmptyBatch) throws Exception { - prepareParquetDFSSource(useSchemaProvider, transformerClassNames != null); + PARQUET_SOURCE_ROOT = dfsBasePath + "/parquetFilesDfs" + testNum; + int parquetRecordsCount = 10; + boolean hasTransformer = transformerClassNames != null && !transformerClassNames.isEmpty(); + prepareParquetDFSFiles(parquetRecordsCount, PARQUET_SOURCE_ROOT, FIRST_PARQUET_FILE_NAME, false, null, null); + prepareParquetDFSSource(useSchemaProvider, hasTransformer, "source.avsc", "target.avsc", PROPS_FILENAME_TEST_PARQUET, + PARQUET_SOURCE_ROOT, false, "partition_path", testEmptyBatch ? "1" : ""); + String tableBasePath = dfsBasePath + "/test_parquet_table" + testNum; HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer( TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, testEmptyBatch ? TestParquetDFSSourceEmptyBatch.class.getName() @@ -1514,22 +1570,38 @@ private void testParquetDFSSource(boolean useSchemaProvider, List transf transformerClassNames, PROPS_FILENAME_TEST_PARQUET, false, useSchemaProvider, 100000, false, null, null, "timestamp", null), jsc); deltaStreamer.sync(); - TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath, sqlContext); - testNum++; + TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath, sqlContext); if (testEmptyBatch) { prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "2.parquet", false, null, null); - // parquet source to return empty batch - TestParquetDFSSourceEmptyBatch.returnEmptyBatch = true; deltaStreamer.sync(); // since we mimic'ed empty batch, total records should be same as first sync(). - TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath, sqlContext); + TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath, sqlContext); HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build(); // validate table schema fetches valid schema from last but one commit. TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); assertNotEquals(tableSchemaResolver.getTableAvroSchema(), Schema.create(Schema.Type.NULL).toString()); } + + // proceed w/ non empty batch. + prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "3.parquet", false, null, null); + deltaStreamer.sync(); + TestHelpers.assertRecordCount(parquetRecordsCount + 100, tableBasePath, sqlContext); + // validate commit metadata for all completed commits to have valid schema in extra metadata. + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build(); + metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().forEach(entry -> assertValidSchemaInCommitMetadata(entry, metaClient)); + testNum++; + } + + private void assertValidSchemaInCommitMetadata(HoodieInstant instant, HoodieTableMetaClient metaClient) { + try { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + assertFalse(StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY))); + } catch (IOException ioException) { + throw new HoodieException("Failed to parse commit metadata for " + instant.toString()); + } } private void testORCDFSSource(boolean useSchemaProvider, List transformerClassNames) throws Exception { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java index 1806d5c48b06d..a5a39dbe2d09e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDataSource.java @@ -39,11 +39,14 @@ public class TestDataSource extends AbstractBaseTestSource { private static final Logger LOG = LogManager.getLogger(TestDataSource.class); + public static boolean returnEmptyBatch = false; + private static int counter = 0; public TestDataSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider) { super(props, sparkContext, sparkSession, schemaProvider); initDataGen(); + returnEmptyBatch = false; } @Override @@ -54,9 +57,13 @@ protected InputBatch> fetchNewData(Option lastChe LOG.info("Source Limit is set to " + sourceLimit); // No new data. - if (sourceLimit <= 0) { + if (sourceLimit <= 0 || returnEmptyBatch) { + LOG.warn("Return no new data from Test Data source " + counter + ", source limit " + sourceLimit); return new InputBatch<>(Option.empty(), lastCheckpointStr.orElse(null)); + } else { + LOG.warn("Returning valid data from Test Data source " + counter + ", source limit " + sourceLimit); } + counter++; List records = fetchNextBatch(props, (int) sourceLimit, instantTime, DEFAULT_PARTITION_NUM).collect(Collectors.toList()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java index 3129e91a9d3e0..11c3f4c8f95ee 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -29,19 +30,28 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + public class TestParquetDFSSourceEmptyBatch extends ParquetDFSSource { - public static boolean returnEmptyBatch; + public static String RETURN_EMPTY_BATCH = "test.dfs.source.return.empty.batches.for"; + public static String DEFAULT_RETURN_EMPTY_BATCH = ""; + public List emptyBatches; + private int counter = 0; public TestParquetDFSSourceEmptyBatch(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider) { super(props, sparkContext, sparkSession, schemaProvider); + String[] emptyBatchesStr = props.getString(RETURN_EMPTY_BATCH, DEFAULT_RETURN_EMPTY_BATCH).split(","); + this.emptyBatches = Arrays.stream(emptyBatchesStr).filter(entry -> !StringUtils.isNullOrEmpty(entry)).map(entry -> Integer.parseInt(entry)).collect(Collectors.toList()); } @Override public Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { Pair>, String> toReturn = super.fetchNextBatch(lastCkptStr, sourceLimit); - if (returnEmptyBatch) { + if (emptyBatches.contains(counter++)) { return Pair.of(Option.empty(), toReturn.getRight()); } return toReturn; diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index a322daaabe9a1..903671d754c76 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -111,6 +111,7 @@ com.github.davidmoten:guava-mini com.github.davidmoten:hilbert-curve + com.github.ben-manes.caffeine:caffeine com.twitter:bijection-avro_${scala.binary.version} com.twitter:bijection-core_${scala.binary.version} io.dropwizard.metrics:metrics-core diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 698cc534d0807..d6a5eb6924618 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -29,6 +29,7 @@ true ${project.parent.basedir} + 3.1.0 @@ -80,6 +81,7 @@ org.apache.hudi:hudi-timeline-service org.apache.hudi:hudi-aws + javax.servlet:javax.servlet-api com.beust:jcommander io.javalin:javalin @@ -138,6 +140,10 @@ + + javax.servlet. + org.apache.hudi.javax.servlet. + org.apache.spark.sql.avro. org.apache.hudi.org.apache.spark.sql.avro. @@ -378,6 +384,12 @@ hive-service ${hive.version} ${spark.bundle.hive.scope} + + + servlet-api + javax.servlet + + @@ -426,6 +438,13 @@ curator-recipes ${zk-curator.version} + + + javax.servlet + javax.servlet-api + ${javax.servlet.version} + + diff --git a/pom.xml b/pom.xml index 7caff57f066b4..9e7b4ab14a559 100644 --- a/pom.xml +++ b/pom.xml @@ -1041,13 +1041,6 @@ - - com.google.code.gson - gson - 2.3.1 - test - - org.apache.curator @@ -1657,6 +1650,7 @@ 1.12.2 1.10.2 1.6.12 + 4.8 ${fasterxml.spark3.version} ${fasterxml.spark3.version} ${fasterxml.spark3.version} @@ -1688,6 +1682,7 @@ hudi-spark3-common ${scalatest.spark3.version} ${kafka.spark3.version} + 4.8-1 ${fasterxml.spark3.version} ${fasterxml.spark3.version} ${fasterxml.spark3.version} @@ -1722,6 +1717,7 @@ 1.12.2 1.10.2 1.6.12 + 4.8 ${fasterxml.spark3.version} ${fasterxml.spark3.version} ${fasterxml.spark3.version} diff --git a/rfc/README.md b/rfc/README.md index 532f38fc5dce3..0ccf7b1bbe285 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -88,3 +88,5 @@ The list of all RFCs can be found here. | 50 | [Improve Timeline Server](./rfc-50/rfc-50.md) | `UNDER REVIEW` | | 51 | [Change Data Capture](./rfc-51/rfc-51.md) | `UNDER REVIEW` | | 52 | [Introduce Secondary Index to Improve HUDI Query Performance](./rfc-52/rfc-52.md) | `UNDER REVIEW` | +| 53 | [Use Lock-Free Message Queue Improving Hoodie Writing Efficiency](./rfc-53/rfc-53.md) | `UNDER REVIEW` | + diff --git a/rfc/rfc-50/ComparisonDiagram.png b/rfc/rfc-50/ComparisonDiagram.png new file mode 100644 index 0000000000000..20fa4673144ec Binary files /dev/null and b/rfc/rfc-50/ComparisonDiagram.png differ diff --git a/rfc/rfc-50/CurrentDesign.png b/rfc/rfc-50/CurrentDesign.png new file mode 100644 index 0000000000000..15e38cb9b1cf1 Binary files /dev/null and b/rfc/rfc-50/CurrentDesign.png differ diff --git a/rfc/rfc-50/Design.png b/rfc/rfc-50/Design.png new file mode 100644 index 0000000000000..86a726c636dab Binary files /dev/null and b/rfc/rfc-50/Design.png differ diff --git a/rfc/rfc-50/SchematicDiagram.png b/rfc/rfc-50/SchematicDiagram.png new file mode 100644 index 0000000000000..9db88decadc4d Binary files /dev/null and b/rfc/rfc-50/SchematicDiagram.png differ diff --git a/rfc/rfc-50/rfc-50.md b/rfc/rfc-50/rfc-50.md new file mode 100644 index 0000000000000..91e205c7dbeff --- /dev/null +++ b/rfc/rfc-50/rfc-50.md @@ -0,0 +1,93 @@ + + +# RFC-50: Improve Timeline Server + +## Proposers +- @yuzhaojing + +## Approvers + - @xushiyan + - @danny0405 + +## Abstract + +Support client to obtain timeline from timeline server. + +## Background + +At its core, Hudi maintains a timeline of all actions performed on the table at different instants of time. Before each operation is performed on the Hoodie table, the information of the HUDI table needs to be obtained through the timeline. At present, there are two ways to obtain the timeline of HUDI : +- Create a MetaClient and get the complete timeline through MetaClient #getActiveTimeline, which will directly scan the HDFS directory of metadata +- Get the timeline through FileSystemView#getTimeline. This timeline is the cache timeline obtained by requesting the Embedded timeline service. There is no need to repeatedly scan the HDFS directory of metadata, but this timeline only contains completed instants + +### Problem description + +- HUDI designs the Timeline service for processing and caching when accessing metadata , but currently does not converge all access to metadata to the Timeline service, such as the acquisition of a complete timeline. +- When the number of tasks written increases, a large number of repeated access to metadata will lead to high HDFS NameNode requests, causing greater pressure and not easy to expand. + +### Spark and Flink write flow comparison diagram + +Since Hudi is designed based on the Spark micro-batch model, in the Spark write process, all operations on the timeline are completed on the driver side, and then distributed to the executor side to start the write operation. + +But for Flink , Write tasks are resident services due to their pure streaming model. There is also no highly reliable communication mechanism between the user-side JM and the TM in Flink, so the TM needs to obtain the latest instant by polling the timeline for writing. + +![](ComparisonDiagram.png) + +### Current + +![](CurrentDesign.png) + +The current design implementation has two main problems with the convergence timeline +- Since the timeline of the task is pulled from the Embedded timeline service, the refresh mechanism of the Embedded timeline service itself will doesn't work +- MetaClient and HoodieTable are decoupled. Obtain the timeline in MetaClient and then request the Embedded timeline service to obtain file-related information through the FileSystemViewManager in HoodieTable combined with the timeline. There are circular dependencies and problems in the case of using MetaClient alone without creating HoodieTable + +## Implementation + +### Design target + +The goal of this solution is to converge the acquisition of timelines and obtain them through the Embedded timeline service uniformly. The timeline is pulled through HDFS only when the Embedded timeline service is not started. + +### Converge the request to loop instant in Flink to JM + +- Store the latest instant on the Embedded Timeline Server. Every time JM modifies the instant state, it actively performs a sync to Embedded Timeline Server +- Return the latest instant directly when the task pulls the latest instant + +### Converge the request to pull instant in meta client initialization to JM + +- Abstract the timeline-related acquisition methods into the new interface TableTimelineView, and create the corresponding TimelineViewManager in MetaClient, and obtain the timeline through TimelineViewManager. + +![](Design.png) + +### Flink optimization before and after schematic diagram + +![](SchematicDiagram.png) + +## Rollout/Adoption Plan + +- What impact (if any) will there be on existing users? + - Since the Embedded Timeline Service is used to pull the timeline, users who use flink to write to hudi will observe that file system requests are greatly reduced, thereby reducing the pressure on the file system. + - However, in a scenario with a relatively high degree of parallelism, it may be necessary to increase the resources of JM to ensure the effectiveness of the response +- If we are changing behavior how will we phase out the older behavior? + - Add a configuration to control this behavior +- If we need special migration tools, describe them here. + - No special migration tools will be necessary +- When will we remove the existing behavior + - In subsequent releases (1.0 or later) +## Test Plan + +Test plan +No additional regression testing is required, as the behavior of MetaClient's active timeline has not been changed