diff --git a/docker/demo/config/test-suite/complex-dag-cow.yaml b/docker/demo/config/test-suite/complex-dag-cow.yaml index a10026c0b948b..5fa8596830f38 100644 --- a/docker/demo/config/test-suite/complex-dag-cow.yaml +++ b/docker/demo/config/test-suite/complex-dag-cow.yaml @@ -13,122 +13,56 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -first_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 1000 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 10000 - deps: first_insert - type: InsertNode -third_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 300 - deps: second_insert - type: InsertNode -first_rollback: - config: - deps: third_insert - type: RollbackNode -first_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_rollback -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 11300 - type: HiveQueryNode - deps: first_hive_sync -second_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_hive_query -second_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 11600 - type: HiveQueryNode - deps: second_upsert -fourth_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 1000 - deps: second_hive_query - type: InsertNode -third_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 12600 - type: HiveQueryNode - deps: fourth_insert -first_delete: - config: - record_size: 70000 - num_partitions_delete: 1 - num_records_delete: 200 - deps: third_hive_query - type: DeleteNode -fourth_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_delete -fourth_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 12400 - type: HiveQueryNode - deps: fourth_hive_sync \ No newline at end of file +dag_name: cow-long-running-example.yaml +dag_rounds: 2 +dag_intermittent_delay_mins: 1 +dag_content: + first_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_validate: + config: + type: ValidateDatasetNode + deps: third_insert + first_upsert: + config: + record_size: 100 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 1 + num_records_delete: 2000 + type: DeleteNode + deps: first_upsert + second_validate: + config: + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/complex-dag-mor.yaml b/docker/demo/config/test-suite/complex-dag-mor.yaml index 2652b03070a7e..505e5e2945f9b 100644 --- a/docker/demo/config/test-suite/complex-dag-mor.yaml +++ b/docker/demo/config/test-suite/complex-dag-mor.yaml @@ -13,103 +13,107 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -first_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 100 - deps: first_insert - type: InsertNode -third_insert: - config: - record_size: 70000 - num_insert_partitions: 1 - repeat_count: 1 - num_records_insert: 300 - deps: second_insert - type: InsertNode -first_rollback: - config: - deps: third_insert - type: RollbackNode -first_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_rollback -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveQueryNode - deps: first_hive_sync -second_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_hive_query -second_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 1100 - type: HiveQueryNode - deps: second_upsert -first_schedule_compact: - config: - type: ScheduleCompactNode - deps: second_hive_query -third_upsert: - config: - record_size: 70000 - num_insert_partitions: 1 - num_records_insert: 300 - repeat_count: 1 - num_records_upsert: 100 - num_upsert_partitions: 10 - type: UpsertNode - deps: first_schedule_compact -first_compact: - config: - type: CompactNode - deps: first_schedule_compact -third_hive_query: - config: - queue_name: "adhoc" - engine: "mr" - hive_queries: - query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" - result1: 0 - query2: "select count(*) from testdb.table1" - result2: 1400 - type: HiveQueryNode - deps: first_compact +dag_name: complex-dag-mor.yaml +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 5 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 5 + num_records_insert: 100 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_rollback: + config: + deps: third_insert + type: RollbackNode + first_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_rollback + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveQueryNode + deps: first_hive_sync + second_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_hive_query + second_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + hive_queries: + query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" + result1: 0 + query2: "select count(*) from testdb.table1" + result2: 1100 + type: HiveQueryNode + deps: second_upsert + first_schedule_compact: + config: + type: ScheduleCompactNode + deps: second_hive_query + third_upsert: + config: + record_size: 70000 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 10 + type: UpsertNode + deps: first_schedule_compact + first_compact: + config: + type: CompactNode + deps: first_schedule_compact + third_hive_query: + config: + queue_name: "adhoc" + engine: "mr" + hive_queries: + query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1" + result1: 0 + query2: "select count(*) from testdb.table1" + result2: 1400 + type: HiveQueryNode + deps: first_compact diff --git a/docker/demo/config/test-suite/cow-long-running-example.yaml b/docker/demo/config/test-suite/cow-long-running-example.yaml new file mode 100644 index 0000000000000..b7026f2dd653a --- /dev/null +++ b/docker/demo/config/test-suite/cow-long-running-example.yaml @@ -0,0 +1,68 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +dag_name: cow-long-running-example.yaml +dag_rounds: 20 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 100 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_validate: + config: + type: ValidateDatasetNode + deps: third_insert + first_upsert: + config: + record_size: 100 + num_partitions_insert: 1 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 1 + num_records_delete: 2000 + type: DeleteNode + deps: first_upsert + second_validate: + config: + delete_input_data: true + type: ValidateDatasetNode + deps: first_delete \ No newline at end of file diff --git a/docker/demo/config/test-suite/test-source.properties b/docker/demo/config/test-suite/test-source.properties deleted file mode 100644 index cc18a39d57086..0000000000000 --- a/docker/demo/config/test-suite/test-source.properties +++ /dev/null @@ -1,37 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -# write configs -hoodie.datasource.write.recordkey.field=_row_key -hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator -hoodie.datasource.write.partitionpath.field=timestamp - - -# deltastreamer configs -hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd -hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP -hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-bench/input -hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/test-suite/source.avsc -hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc - -#hive sync -hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/ -hoodie.datasource.hive_sync.database=testdb -hoodie.datasource.hive_sync.table=table1 -hoodie.datasource.hive_sync.use_jdbc=false -hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor -hoodie.datasource.hive_sync.assume_date_partitioning=true -hoodie.datasource.hive_sync.use_pre_apache_input_format=true diff --git a/docker/demo/config/test-suite/test.properties b/docker/demo/config/test-suite/test.properties index a7fd3986a730b..0aa0f45c05ed5 100644 --- a/docker/demo/config/test-suite/test.properties +++ b/docker/demo/config/test-suite/test.properties @@ -1,3 +1,4 @@ + hoodie.insert.shuffle.parallelism=100 hoodie.upsert.shuffle.parallelism=100 hoodie.bulkinsert.shuffle.parallelism=100 @@ -8,6 +9,13 @@ hoodie.deltastreamer.source.test.max_unique_records=100000000 hoodie.embed.timeline.server=false hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.insert.shuffle.parallelism=100 +hoodie.upsert.shuffle.parallelism=100 +hoodie.bulkinsert.shuffle.parallelism=100 + +hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector +hoodie.datasource.hive_sync.skip_ro_suffix=true + hoodie.datasource.write.recordkey.field=_row_key hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator hoodie.datasource.write.partitionpath.field=timestamp diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index febfcd28a1116..6409cc747a4be 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -21,6 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.spark.sql.SaveMode; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.HoodieDataSourceHelpers; +import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hadoop.fs.FileSystem; val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) @@ -49,6 +50,8 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive"). option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true"). option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr"). + option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor"); @@ -79,6 +82,8 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive"). option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true"). option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr"). + option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName). + option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor_bs"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index 2c6be7ff03e91..014998749ca3f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -21,6 +21,7 @@ import com.codahale.metrics.Timer; import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -38,9 +39,12 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -88,6 +92,7 @@ public abstract class AbstractHoodieWriteClient table, HoodieCommitMetadata me } else { metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false"); } + + // Do an inline clustering if enabled + if (config.isInlineClustering()) { + runAnyPendingClustering(table); + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true"); + inlineCluster(extraMetadata); + } else { + metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false"); + } + // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table); archiveLog.archiveIfRequired(context); @@ -406,6 +421,16 @@ protected void runAnyPendingCompactions(HoodieTable table) { }); } + protected void runAnyPendingClustering(HoodieTable table) { + table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> { + Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); + if (instantPlan.isPresent()) { + LOG.info("Running pending clustering at instant " + instantPlan.get().getLeft()); + cluster(instant.getTimestamp(), true); + } + }); + } + /** * Handle auto clean during commit. * @@ -574,11 +599,6 @@ public HoodieCleanMetadata clean() { * Provides a new commit time for a write operation (insert/update/delete). */ public String startCommit() { - // NOTE : Need to ensure that rollback is done before a new commit is started - if (rollbackPending) { - // Only rollback pending commit/delta-commits. Do not touch compaction commits - rollbackPendingCommits(); - } String instantTime = HoodieActiveTimeline.createNewInstantTime(); startCommitWithTime(instantTime); return instantTime; @@ -725,6 +745,54 @@ protected Option inlineCompact(Option> extraMetadata return compactionInstantTimeOpt; } + /** + * Schedules a new clustering instant. + * + * @param extraMetadata Extra Metadata to be stored + */ + public Option scheduleClustering(Option> extraMetadata) throws HoodieIOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleClusteringAtInstant(instantTime, extraMetadata) ? Option.of(instantTime) : Option.empty(); + } + + /** + * Schedules a new clustering instant with passed-in instant time. + * + * @param instantTime clustering Instant Time + * @param extraMetadata Extra Metadata to be stored + */ + public boolean scheduleClusteringAtInstant(String instantTime, Option> extraMetadata) throws HoodieIOException { + LOG.info("Scheduling clustering at instant time :" + instantTime); + Option plan = createTable(config, hadoopConf) + .scheduleClustering(context, instantTime, extraMetadata); + return plan.isPresent(); + } + + /** + * Ensures clustering instant is in expected state and performs clustering for the plan stored in metadata. + * + * @param clusteringInstant Clustering Instant Time + * @return Collection of Write Status + */ + public abstract HoodieWriteMetadata cluster(String clusteringInstant, boolean shouldComplete); + + /** + * Executes a clustering plan on a table, serially before or after an insert/upsert action. + */ + protected Option inlineCluster(Option> extraMetadata) { + Option clusteringInstantOpt = scheduleClustering(extraMetadata); + clusteringInstantOpt.ifPresent(clusteringInstant -> { + // inline cluster should auto commit as the user is never given control + cluster(clusteringInstant, true); + }); + return clusteringInstantOpt; + } + + protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false); + table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant); + } + /** * Finalize Write operation. * @@ -772,7 +840,8 @@ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { try { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); Option lastInstant = - activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType())) + activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType()) + || s.getAction().equals(HoodieActiveTimeline.REPLACE_COMMIT_ACTION)) .lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java new file mode 100644 index 0000000000000..aa6c29b0844a2 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ConcatenatingIterator.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.common.util.ValidationUtils; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +/** + * Provides iterator interface over List of iterators. Consumes all records from first iterator element + * before moving to next iterator in the list. That is concatenate elements across multiple iterators. + * + * @param + */ +public class ConcatenatingIterator implements Iterator { + + private final Queue> allIterators; + + public ConcatenatingIterator(List> iterators) { + allIterators = new LinkedList<>(iterators); + } + + @Override + public boolean hasNext() { + while (!allIterators.isEmpty()) { + if (allIterators.peek().hasNext()) { + return true; + } + // iterator at current head is done. move ahead + allIterators.poll(); + } + + return false; + } + + @Override + public T next() { + ValidationUtils.checkArgument(hasNext(), "No more elements left"); + return allIterators.peek().next(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java new file mode 100644 index 0000000000000..347f86cfba349 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/FileSliceMetricUtils.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieLogFile; + +import java.util.List; +import java.util.Map; + +/** + * A utility class for calculating metrics related to FileSlice. + */ +public class FileSliceMetricUtils { + + public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB"; + public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB"; + public static final String TOTAL_IO_MB = "TOTAL_IO_MB"; + public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE"; + public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES"; + + public static void addFileSliceCommonMetrics(List fileSlices, Map metrics, long defaultBaseFileSize) { + int numLogFiles = 0; + long totalLogFileSize = 0; + long totalIORead = 0; + long totalIOWrite = 0; + long totalIO = 0; + + for (FileSlice slice : fileSlices) { + numLogFiles += slice.getLogFiles().count(); + // Total size of all the log files + totalLogFileSize += slice.getLogFiles().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) + .reduce(Long::sum).orElse(0L); + + long baseFileSize = slice.getBaseFile().isPresent() ? slice.getBaseFile().get().getFileSize() : 0L; + // Total read will be the base file + all the log files + totalIORead = FSUtils.getSizeInMB(baseFileSize + totalLogFileSize); + // Total write will be similar to the size of the base file + totalIOWrite = FSUtils.getSizeInMB(baseFileSize > 0 ? baseFileSize : defaultBaseFileSize); + // Total IO will the the IO for read + write + totalIO = totalIORead + totalIOWrite; + } + + metrics.put(TOTAL_IO_READ_MB, (double) totalIORead); + metrics.put(TOTAL_IO_WRITE_MB, (double) totalIOWrite); + metrics.put(TOTAL_IO_MB, (double) totalIO); + metrics.put(TOTAL_LOG_FILE_SIZE, (double) totalLogFileSize); + metrics.put(TOTAL_LOG_FILES, (double) numLogFiles); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java new file mode 100644 index 0000000000000..91acd3075377a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.config; + +import org.apache.hudi.common.config.DefaultHoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * Clustering specific configs. + */ +public class HoodieClusteringConfig extends DefaultHoodieConfig { + + // Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy + public static final String CLUSTERING_PLAN_STRATEGY_CLASS = "hoodie.clustering.plan.strategy.class"; + public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS = + "org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy"; + + // Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy + public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = "hoodie.clustering.execution.strategy.class"; + public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS = + "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy"; + + // Turn on inline clustering - clustering will be run after write operation is complete. + public static final String INLINE_CLUSTERING_PROP = "hoodie.clustering.inline"; + private static final String DEFAULT_INLINE_CLUSTERING = "false"; + + // Config to control frequency of clustering + public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = "hoodie.clustering.inline.max.commits"; + private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = "4"; + + // Any strategy specific params can be saved with this prefix + public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy."; + + // Number of partitions to list to create ClusteringPlan. + public static final String CLUSTERING_TARGET_PARTITIONS = CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions"; + public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = String.valueOf(2); + + // Files smaller than the size specified here are candidates for clustering. + public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit"; + public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = String.valueOf(600 * 1024 * 1024L); // 600MB + + // Each clustering operation can create multiple groups. Total amount of data processed by clustering operation + // is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS). + // Max amount of data to be included in one group + public static final String CLUSTERING_MAX_BYTES_PER_GROUP = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group"; + public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = String.valueOf(2 * 1024 * 1024 * 1024L); + + // Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. + public static final String CLUSTERING_MAX_NUM_GROUPS = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups"; + public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = "30"; + + // Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups. + public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes"; + public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = String.valueOf(1 * 1024 * 1024 * 1024L); // 1GB + + // Constants related to clustering that may be used by more than 1 strategy. + public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns"; + + // When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update + public static final String CLUSTERING_UPDATES_STRATEGY_PROP = "hoodie.clustering.updates.strategy"; + public static final String DEFAULT_CLUSTERING_UPDATES_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy"; + + // Async clustering + public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = "hoodie.clustering.async.enabled"; + public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = "false"; + + public HoodieClusteringConfig(Properties props) { + super(props); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private final Properties props = new Properties(); + + public Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + this.props.load(reader); + return this; + } + } + + public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) { + props.setProperty(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass); + return this; + } + + public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) { + props.setProperty(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass); + return this; + } + + public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) { + props.setProperty(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions)); + return this; + } + + public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) { + props.setProperty(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit)); + return this; + } + + public Builder withClusteringSortColumns(String sortColumns) { + props.setProperty(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns); + return this; + } + + public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) { + props.setProperty(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize)); + return this; + } + + public Builder withClusteringMaxNumGroups(int maxNumGroups) { + props.setProperty(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups)); + return this; + } + + public Builder withClusteringTargetFileMaxBytes(long targetFileSize) { + props.setProperty(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize)); + return this; + } + + public Builder withInlineClustering(Boolean inlineClustering) { + props.setProperty(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering)); + return this; + } + + public Builder withInlineClusteringNumCommits(int numCommits) { + props.setProperty(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits)); + return this; + } + + public Builder fromProperties(Properties props) { + this.props.putAll(props); + return this; + } + + public Builder withClusteringUpdatesStrategy(String updatesStrategyClass) { + props.setProperty(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass); + return this; + } + + public Builder withAsyncClustering(Boolean asyncClustering) { + props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering)); + return this; + } + + public HoodieClusteringConfig build() { + HoodieClusteringConfig config = new HoodieClusteringConfig(props); + + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_STRATEGY_CLASS), + CLUSTERING_PLAN_STRATEGY_CLASS, DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_EXECUTION_STRATEGY_CLASS), + CLUSTERING_EXECUTION_STRATEGY_CLASS, DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_BYTES_PER_GROUP), CLUSTERING_MAX_BYTES_PER_GROUP, + DEFAULT_CLUSTERING_MAX_GROUP_SIZE); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_NUM_GROUPS), CLUSTERING_MAX_NUM_GROUPS, + DEFAULT_CLUSTERING_MAX_NUM_GROUPS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_FILE_MAX_BYTES), CLUSTERING_TARGET_FILE_MAX_BYTES, + DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES); + setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_PROP), INLINE_CLUSTERING_PROP, + DEFAULT_INLINE_CLUSTERING); + setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_MAX_COMMIT_PROP), INLINE_CLUSTERING_MAX_COMMIT_PROP, + DEFAULT_INLINE_CLUSTERING_NUM_COMMITS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_PARTITIONS), CLUSTERING_TARGET_PARTITIONS, + DEFAULT_CLUSTERING_TARGET_PARTITIONS); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_SMALL_FILE_LIMIT), CLUSTERING_PLAN_SMALL_FILE_LIMIT, + DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT); + setDefaultOnCondition(props, !props.containsKey(CLUSTERING_UPDATES_STRATEGY_PROP), CLUSTERING_UPDATES_STRATEGY_PROP, + DEFAULT_CLUSTERING_UPDATES_STRATEGY); + setDefaultOnCondition(props, !props.containsKey(ASYNC_CLUSTERING_ENABLE_OPT_KEY), ASYNC_CLUSTERING_ENABLE_OPT_KEY, + DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL); + return config; + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index e5baaf6ec6762..18bb4b3a93d45 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 @@ -50,6 +50,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; + /** * Class storing configs for the HoodieWriteClient. */ @@ -70,6 +71,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { public static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism"; public static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism"; public static final String BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = "hoodie.bulkinsert.user.defined.partitioner.class"; + public static final String BULKINSERT_INPUT_DATA_SCHEMA_DDL = "hoodie.bulkinsert.schema.ddl"; public static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism"; public static final String DELETE_PARALLELISM = "hoodie.delete.shuffle.parallelism"; public static final String DEFAULT_ROLLBACK_PARALLELISM = "100"; @@ -390,6 +392,23 @@ public Boolean getCompactionReverseLogReadEnabled() { return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP)); } + public boolean isInlineClustering() { + return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP)); + } + + public boolean isAsyncClusteringEnabled() { + return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY)); + } + + public boolean isClusteringEnabled() { + // TODO: future support async clustering + return isInlineClustering() || isAsyncClusteringEnabled(); + } + + public int getInlineClusterMaxCommits() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP)); + } + public String getPayloadClass() { return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP); } @@ -406,6 +425,45 @@ public Boolean shouldCleanBootstrapBaseFile() { return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED)); } + public String getClusteringUpdatesStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP); + } + + /** + * Clustering properties. + */ + public String getClusteringPlanStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS); + } + + public String getClusteringExecutionStrategyClass() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS); + } + + public long getClusteringMaxBytesInGroup() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP)); + } + + public long getClusteringSmallFileLimit() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT)); + } + + public int getClusteringMaxNumGroups() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS)); + } + + public long getClusteringTargetFileMaxBytes() { + return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES)); + } + + public int getTargetPartitionsForClustering() { + return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS)); + } + + public String getClusteringSortColumns() { + return props.getProperty(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY); + } + /** * index properties. */ @@ -804,6 +862,7 @@ public static class Builder { private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; + private boolean isClusteringConfigSet = false; private boolean isMetricsConfigSet = false; private boolean isBootstrapConfigSet = false; private boolean isMemoryConfigSet = false; @@ -933,6 +992,12 @@ public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) { return this; } + public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) { + props.putAll(clusteringConfig.getProps()); + isClusteringConfigSet = true; + return this; + } + public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) { props.putAll(metricsConfig.getProps()); isMetricsConfigSet = true; @@ -1087,6 +1152,8 @@ protected void setDefaults() { setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isCompactionConfigSet, HoodieCompactionConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !isClusteringConfigSet, + HoodieClusteringConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isBootstrapConfigSet, HoodieBootstrapConfig.newBuilder().fromProperties(props).build()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java new file mode 100644 index 0000000000000..bb6aaa24777cf --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +public class HoodieClusteringException extends HoodieException { + + public HoodieClusteringException(String msg) { + super(msg); + } + + public HoodieClusteringException(String msg, Throwable e) { + super(msg, e); + } +} + diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java new file mode 100644 index 0000000000000..68b62a5421706 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieClusteringUpdateException.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.exception; + +public class HoodieClusteringUpdateException extends HoodieException { + public HoodieClusteringUpdateException(String msg) { + super(msg); + } + + public HoodieClusteringUpdateException(String msg, Throwable e) { + super(msg, e); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java index 292039b83ad41..c8c112fc203ff 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java @@ -37,6 +37,7 @@ public class HoodieMetrics { public String cleanTimerName = null; public String commitTimerName = null; public String deltaCommitTimerName = null; + public String replaceCommitTimerName = null; public String finalizeTimerName = null; public String compactionTimerName = null; public String indexTimerName = null; @@ -48,6 +49,7 @@ public class HoodieMetrics { private Timer deltaCommitTimer = null; private Timer finalizeTimer = null; private Timer compactionTimer = null; + private Timer clusteringTimer = null; private Timer indexTimer = null; public HoodieMetrics(HoodieWriteConfig config, String tableName) { @@ -59,6 +61,7 @@ public HoodieMetrics(HoodieWriteConfig config, String tableName) { this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION); this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION); this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION); + this.replaceCommitTimerName = getMetricsName("timer", HoodieTimeline.REPLACE_COMMIT_ACTION); this.finalizeTimerName = getMetricsName("timer", "finalize"); this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION); this.indexTimerName = getMetricsName("timer", "index"); @@ -83,6 +86,13 @@ public Timer.Context getCompactionCtx() { return compactionTimer == null ? null : compactionTimer.time(); } + public Timer.Context getClusteringCtx() { + if (config.isMetricsOn() && clusteringTimer == null) { + clusteringTimer = createTimer(replaceCommitTimerName); + } + return clusteringTimer == null ? null : clusteringTimer.time(); + } + public Timer.Context getCleanCtx() { if (config.isMetricsOn() && cleanTimer == null) { cleanTimer = createTimer(cleanTimerName); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 644eb6273d093..6b7a7d29cc436 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -157,6 +158,15 @@ public abstract HoodieWriteMetadata bulkInsert(HoodieEngineContext context, S */ public abstract HoodieWriteMetadata delete(HoodieEngineContext context, String instantTime, K keys); + /** + * Deletes all data of partitions. + * @param context HoodieEngineContext + * @param instantTime Instant Time for the action + * @param partitions {@link List} of partition to be deleted + * @return HoodieWriteMetadata + */ + public abstract HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions); + /** * Upserts the given prepared records into the Hoodie table, at the supplied instantTime. *

@@ -337,6 +347,27 @@ public abstract Option scheduleCompaction(HoodieEngineCont public abstract HoodieWriteMetadata compact(HoodieEngineContext context, String compactionInstantTime); + + /** + * Schedule clustering for the instant time. + * + * @param context HoodieEngineContext + * @param instantTime Instant Time for scheduling clustering + * @param extraMetadata additional metadata to write into plan + * @return HoodieClusteringPlan, if there is enough data for clustering. + */ + public abstract Option scheduleClustering(HoodieEngineContext context, + String instantTime, + Option> extraMetadata); + + /** + * Execute Clustering on the table. Clustering re-arranges the data so that it is optimized for data access. + * + * @param context HoodieEngineContext + * @param clusteringInstantTime Instant Time + */ + public abstract HoodieWriteMetadata cluster(HoodieEngineContext context, String clusteringInstantTime); + /** * Perform metadata/full bootstrap of a Hudi table. * @param context HoodieEngineContext diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java new file mode 100644 index 0000000000000..c8329b87f86c9 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/BaseClusteringPlanActionExecutor.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + +public abstract class BaseClusteringPlanActionExecutor extends BaseActionExecutor> { + + private final Option> extraMetadata; + + public BaseClusteringPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime); + this.extraMetadata = extraMetadata; + } + + protected abstract Option createClusteringPlan(); + + @Override + public Option execute() { + Option planOption = createClusteringPlan(); + if (planOption.isPresent()) { + HoodieInstant clusteringInstant = + new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); + try { + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setOperationType(WriteOperationType.CLUSTER.name()) + .setExtraMetadata(extraMetadata.orElse(Collections.emptyMap())) + .setClusteringPlan(planOption.get()) + .build(); + table.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, + TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + } catch (IOException ioe) { + throw new HoodieIOException("Exception scheduling clustering", ioe); + } + } + return planOption; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java new file mode 100644 index 0000000000000..27a5b23e2f027 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.avro.Schema; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; +import java.util.Map; + +/** + * Pluggable implementation for writing data into new file groups based on ClusteringPlan. + */ +public abstract class ClusteringExecutionStrategy implements Serializable { + private static final Logger LOG = LogManager.getLogger(ClusteringExecutionStrategy.class); + + private final HoodieTable hoodieTable; + private final HoodieEngineContext engineContext; + private final HoodieWriteConfig writeConfig; + + public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + this.hoodieTable = table; + this.engineContext = engineContext; + } + + /** + * Execute clustering to write inputRecords into new files as defined by rules in strategy parameters. The number of new + * file groups created is bounded by numOutputGroups. + * Note that commit is not done as part of strategy. commit is callers responsibility. + */ + public abstract O performClustering(final I inputRecords, final int numOutputGroups, final String instantTime, + final Map strategyParams, final Schema schema); + + protected HoodieTable getHoodieTable() { + return this.hoodieTable; + } + + protected HoodieEngineContext getEngineContext() { + return this.engineContext; + } + + protected HoodieWriteConfig getWriteConfig() { + return this.writeConfig; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java new file mode 100644 index 0000000000000..31f566cb32fa7 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringPlanStrategy.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.utils.FileSliceMetricUtils; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Pluggable implementation for scheduling clustering and creating ClusteringPlan. + */ +public abstract class ClusteringPlanStrategy implements Serializable { + private static final Logger LOG = LogManager.getLogger(ClusteringPlanStrategy.class); + + public static final int CLUSTERING_PLAN_VERSION_1 = 1; + + private final HoodieTable hoodieTable; + private final transient HoodieEngineContext engineContext; + private final HoodieWriteConfig writeConfig; + + public ClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + this.hoodieTable = table; + this.engineContext = engineContext; + } + + /** + * Generate metadata for grouping eligible files and create a plan. Note that data is not moved around + * as part of this step. + * + * If there is no data available to cluster, return None. + */ + public abstract Option generateClusteringPlan(); + + /** + * Return file slices eligible for clustering. FileIds in pending clustering/compaction are not eligible for clustering. + */ + protected Stream getFileSlicesEligibleForClustering(String partition) { + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) getHoodieTable().getSliceView(); + Set fgIdsInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + fgIdsInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet())); + + return hoodieTable.getSliceView().getLatestFileSlices(partition) + // file ids already in clustering are not eligible + .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())); + } + + /** + * Get parameters specific to strategy. These parameters are passed from 'schedule clustering' step to + * 'execute clustering' step. 'execute clustering' step is typically async. So these params help with passing any required + * context from schedule to run step. + */ + protected abstract Map getStrategyParams(); + + /** + * Returns any specific parameters to be stored as part of clustering metadata. + */ + protected Map getExtraMetadata() { + return Collections.emptyMap(); + } + + /** + * Version to support future changes for plan. + */ + protected int getPlanVersion() { + return CLUSTERING_PLAN_VERSION_1; + } + + /** + * Transform {@link FileSlice} to {@link HoodieSliceInfo}. + */ + protected static List getFileSliceInfo(List slices) { + return slices.stream().map(slice -> new HoodieSliceInfo().newBuilder() + .setPartitionPath(slice.getPartitionPath()) + .setFileId(slice.getFileId()) + .setDataFilePath(slice.getBaseFile().map(BaseFile::getPath).orElse(StringUtils.EMPTY_STRING)) + .setDeltaFilePaths(slice.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toList())) + .setBootstrapFilePath(slice.getBaseFile().map(bf -> bf.getBootstrapBaseFile().map(bbf -> bbf.getPath()).orElse(StringUtils.EMPTY_STRING)).orElse(StringUtils.EMPTY_STRING)) + .build()).collect(Collectors.toList()); + } + + /** + * Generate metrics for the data to be clustered. + */ + protected Map buildMetrics(List fileSlices) { + Map metrics = new HashMap<>(); + FileSliceMetricUtils.addFileSliceCommonMetrics(fileSlices, metrics, getWriteConfig().getParquetMaxFileSize()); + return metrics; + } + + protected HoodieTable getHoodieTable() { + return this.hoodieTable; + } + + protected HoodieEngineContext getEngineContext() { + return this.engineContext; + } + + protected HoodieWriteConfig getWriteConfig() { + return this.writeConfig; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java new file mode 100644 index 0000000000000..404cc0258e242 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieClusteringStrategy; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Scheduling strategy with restriction that clustering groups can only contain files from same partition. + */ +public abstract class PartitionAwareClusteringPlanStrategy extends ClusteringPlanStrategy { + private static final Logger LOG = LogManager.getLogger(PartitionAwareClusteringPlanStrategy.class); + + public PartitionAwareClusteringPlanStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + /** + * Create Clustering group based on files eligible for clustering in the partition. + */ + protected abstract Stream buildClusteringGroupsForPartition(String partitionPath, + List fileSlices); + + /** + * Return list of partition paths to be considered for clustering. + */ + protected List filterPartitionPaths(List partitionPaths) { + return partitionPaths; + } + + @Override + public Option generateClusteringPlan() { + try { + HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); + LOG.info("Scheduling clustering for " + metaClient.getBasePath()); + List partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), + getWriteConfig().shouldAssumeDatePartitioning()); + + // filter the partition paths if needed to reduce list status + partitionPaths = filterPartitionPaths(partitionPaths); + + if (partitionPaths.isEmpty()) { + // In case no partitions could be picked, return no clustering plan + return Option.empty(); + } + + List clusteringGroups = getEngineContext().flatMap(partitionPaths, + partitionPath -> { + List fileSlicesEligible = getFileSlicesEligibleForClustering(partitionPath).collect(Collectors.toList()); + return buildClusteringGroupsForPartition(partitionPath, fileSlicesEligible).limit(getWriteConfig().getClusteringMaxNumGroups()); + }, + partitionPaths.size()) + .stream().limit(getWriteConfig().getClusteringMaxNumGroups()).collect(Collectors.toList()); + + if (clusteringGroups.isEmpty()) { + LOG.info("No data available to cluster"); + return Option.empty(); + } + + HoodieClusteringStrategy strategy = HoodieClusteringStrategy.newBuilder() + .setStrategyClassName(getWriteConfig().getClusteringExecutionStrategyClass()) + .setStrategyParams(getStrategyParams()) + .build(); + + return Option.of(HoodieClusteringPlan.newBuilder() + .setStrategy(strategy) + .setInputGroups(clusteringGroups) + .setExtraMetadata(getExtraMetadata()) + .setVersion(getPlanVersion()) + .build()); + } catch (IOException e) { + throw new HoodieIOException("Unable to create clustering plan", e); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java new file mode 100644 index 0000000000000..667a58b355781 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster.strategy; + +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecordPayload; + +import java.util.Set; + +/** + * When file groups in clustering, write records to these file group need to check. + */ +public abstract class UpdateStrategy, I> { + + protected final HoodieEngineContext engineContext; + protected Set fileGroupsInPendingClustering; + + protected UpdateStrategy(HoodieEngineContext engineContext, Set fileGroupsInPendingClustering) { + this.engineContext = engineContext; + this.fileGroupsInPendingClustering = fileGroupsInPendingClustering; + } + + /** + * Check the update records to the file group in clustering. + * @param taggedRecordsRDD the records to write, tagged with target file id, + * future can update tagged records location to a different fileId. + * @return the recordsRDD strategy updated + */ + public abstract I handleUpdate(I taggedRecordsRDD); + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java index 3ead7a07d9698..d2aa8627bd275 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java @@ -27,8 +27,21 @@ public abstract class AbstractBulkInsertHelper { + /** + * Mark instant as inflight, write input records, update index and return result. + */ public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, BaseCommitActionExecutor executor, boolean performDedupe, Option> userDefinedBulkInsertPartitioner); + + /** + * Only write input records. Does not change timeline/index. Return information about new files created. + */ + public abstract O bulkInsert(I inputRecords, String instantTime, + HoodieTable table, HoodieWriteConfig config, + boolean performDedupe, + Option> userDefinedBulkInsertPartitioner, + boolean addMetadataFields, + int parallelism); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java index 6c631c462bda1..ff2dfbd4c1ecb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java @@ -20,14 +20,13 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.util.CompactionUtils; -import org.apache.hudi.common.util.Option; +import org.apache.hudi.client.utils.FileSliceMetricUtils; import org.apache.hudi.config.HoodieWriteConfig; import java.io.Serializable; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -50,32 +49,14 @@ public abstract class CompactionStrategy implements Serializable { * Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the metrics they need * to decide on the priority. * - * @param dataFile - Base file to compact - * @param partitionPath - Partition path - * @param logFiles - List of log files to compact with the base file + * @param writeConfig write configuration. + * @param slice fileSlice to capture metrics for. * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieWriteConfig writeConfig, Option dataFile, - String partitionPath, List logFiles) { + public Map captureMetrics(HoodieWriteConfig writeConfig, FileSlice slice) { Map metrics = new HashMap<>(); long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); - // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) - .reduce(Long::sum).orElse(0L); - // Total read will be the base file + all the log files - Long totalIORead = - FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L) + totalLogFileSize); - // Total write will be similar to the size of the base file - Long totalIOWrite = - FSUtils.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize); - // Total IO will the the IO for read + write - long totalIO = totalIORead + totalIOWrite; - // Save these metrics and we will use during the filter - metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue()); - metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue()); - metrics.put(TOTAL_IO_MB, (double) totalIO); - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); - metrics.put(TOTAL_LOG_FILES, (double) logFiles.size()); + FileSliceMetricUtils.addFileSliceCommonMetrics(Collections.singletonList(slice), metrics, defaultMaxParquetFileSize); return metrics; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java index c9a811a1cceeb..fe4b474593886 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -20,14 +20,10 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import java.util.Comparator; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; /** @@ -40,21 +36,6 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements Comparator { - private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; - - @Override - public Map captureMetrics(HoodieWriteConfig config, Option dataFile, - String partitionPath, List logFiles) { - Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); - - // Total size of all the log files - Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0) - .reduce(Long::sum).orElse(0L); - // save the metrics needed during the order - metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue()); - return metrics; - } - @Override public List orderAndFilter(HoodieWriteConfig writeConfig, List operations, List pendingCompactionPlans) { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java new file mode 100644 index 0000000000000..af4c4fbfc0cb4 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestConcatenatingIterator.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utils; + +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.fail; + +public class TestConcatenatingIterator { + + // Simple test for iterator concatenation + @Test + public void testConcatBasic() { + Iterator i1 = Arrays.asList(5, 3, 2, 1).iterator(); + Iterator i2 = new ArrayList().iterator(); // empty iterator + Iterator i3 = Arrays.asList(3).iterator(); + + ConcatenatingIterator ci = new ConcatenatingIterator<>(Arrays.asList(i1, i2, i3)); + List allElements = new ArrayList<>(); + while (ci.hasNext()) { + allElements.add(ci.next()); + } + + assertEquals(5, allElements.size()); + assertEquals(Arrays.asList(5, 3, 2, 1, 3), allElements); + } + + @Test + public void testConcatError() { + Iterator i1 = new ArrayList().iterator(); // empty iterator + + ConcatenatingIterator ci = new ConcatenatingIterator<>(Arrays.asList(i1)); + assertFalse(ci.hasNext()); + try { + ci.next(); + fail("expected error for empty iterator"); + } catch (IllegalArgumentException e) { + // + } + } +} \ No newline at end of file diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index f975406e4505b..3e8952e0c4439 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 @@ -188,6 +188,11 @@ protected List compact(String compactionInstantTime, boolean should throw new HoodieNotSupportedException("Compaction is not supported yet"); } + @Override + public HoodieWriteMetadata> cluster(final String clusteringInstant, final boolean shouldComplete) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + @Override protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index b8ae370f129d4..d0cb8dec788c9 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -83,6 +84,11 @@ public HoodieWriteMetadata> delete(HoodieEngineContext context return new FlinkDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute(); } + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + throw new HoodieNotSupportedException("DeletePartitions is not supported yet"); + } + @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords) { return new FlinkUpsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute(); @@ -121,6 +127,16 @@ public HoodieWriteMetadata> compact(HoodieEngineContext contex throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } + @Override + public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { + throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + } + + @Override + public HoodieWriteMetadata> cluster(final HoodieEngineContext context, final String clusteringInstantTime) { + throw new HoodieNotSupportedException("Clustering is not supported on a Flink CopyOnWrite table"); + } + @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { throw new HoodieNotSupportedException("Bootstrap is not supported yet"); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 67a6071599665..71a85deff9c30 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -212,6 +212,11 @@ protected List compact(String compactionInstantTime, throw new HoodieNotSupportedException("Compact is not supported in HoodieJavaClient"); } + @Override + public HoodieWriteMetadata> cluster(final String clusteringInstant, final boolean shouldComplete) { + throw new HoodieNotSupportedException("Cluster is not supported in HoodieJavaClient"); + } + @Override protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index 7c45b75fb223f..ddc995ab14de8 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -83,6 +84,11 @@ public HoodieWriteMetadata> delete(HoodieEngineContext context throw new HoodieNotSupportedException("Delete is not supported yet"); } + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + throw new HoodieNotSupportedException("Delete partitions is not supported yet"); + } + @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, @@ -135,6 +141,16 @@ public HoodieWriteMetadata> compact(HoodieEngineContext contex throw new HoodieNotSupportedException("Compact is not supported yet"); } + @Override + public Option scheduleClustering(final HoodieEngineContext context, final String instantTime, final Option> extraMetadata) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + + @Override + public HoodieWriteMetadata> cluster(final HoodieEngineContext context, final String clusteringInstantTime) { + throw new HoodieNotSupportedException("Clustering is not supported yet"); + } + @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { 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 5010a5f76e27e..f7e7690bb163f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -18,6 +18,8 @@ package org.apache.hudi.client; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.common.HoodieEngineContext; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; @@ -25,6 +27,7 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -34,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndex; @@ -43,15 +47,13 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.compact.SparkCompactHelpers; import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade; - -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.List; import java.util.Map; @@ -243,6 +245,13 @@ public JavaRDD delete(JavaRDD keys, String instantTime) return postWrite(result, instantTime, table); } + public HoodieWriteResult deletePartitions(List partitions, String instantTime) { + HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime); + setOperationType(WriteOperationType.DELETE_PARTITION); + HoodieWriteMetadata> result = table.deletePartitions(context,instantTime, partitions); + return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + } + @Override protected JavaRDD postWrite(HoodieWriteMetadata> result, String instantTime, @@ -314,6 +323,57 @@ protected JavaRDD compact(String compactionInstantTime, boolean sho return statuses; } + @Override + public HoodieWriteMetadata> cluster(String clusteringInstant, boolean shouldComplete) { + HoodieSparkTable table = HoodieSparkTable.create(config, context); + HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline(); + HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant); + if (pendingClusteringTimeline.containsInstant(inflightInstant)) { + rollbackInflightClustering(inflightInstant, table); + table.getMetaClient().reloadActiveTimeline(); + } + clusteringTimer = metrics.getClusteringCtx(); + LOG.info("Starting clustering at " + clusteringInstant); + HoodieWriteMetadata> clusteringMetadata = table.cluster(context, clusteringInstant); + JavaRDD statuses = clusteringMetadata.getWriteStatuses(); + if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) { + completeClustering((HoodieReplaceCommitMetadata) clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant); + } + return clusteringMetadata; + } + + protected void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD writeStatuses, + HoodieTable>, JavaRDD, JavaRDD> table, + String clusteringCommitTime) { + + List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + if (!writeStatuses.filter(WriteStatus::hasErrors).isEmpty()) { + throw new HoodieClusteringException("Clustering failed to write to files:" + + writeStatuses.filter(WriteStatus::hasErrors).map(WriteStatus::getFileId).collect()); + } + finalizeWrite(table, clusteringCommitTime, writeStats); + try { + LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata); + table.getActiveTimeline().transitionReplaceInflightToComplete( + HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException e) { + throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); + } + + if (clusteringTimer != null) { + long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); + try { + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(clusteringCommitTime).getTime(), + durationInMs, metadata, HoodieActiveTimeline.REPLACE_COMMIT_ACTION); + } catch (ParseException e) { + throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " + + config.getBasePath() + " at time " + clusteringCommitTime, e); + } + } + LOG.info("Clustering successfully on commit " + clusteringCommitTime); + } + @Override protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType, String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java new file mode 100644 index 0000000000000..f1382ac3c2a33 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.plan.strategy; + +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY; + +/** + * Clustering Strategy based on following. + * 1) Only looks at latest 'daybased.lookback.partitions' partitions. + * 2) Excludes files that are greater than 'small.file.limit' from clustering plan. + */ +public class SparkRecentDaysClusteringPlanStrategy> + extends PartitionAwareClusteringPlanStrategy>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(SparkRecentDaysClusteringPlanStrategy.class); + + public SparkRecentDaysClusteringPlanStrategy(HoodieSparkCopyOnWriteTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public SparkRecentDaysClusteringPlanStrategy(HoodieSparkMergeOnReadTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + protected Stream buildClusteringGroupsForPartition(String partitionPath, List fileSlices) { + List, Integer>> fileSliceGroups = new ArrayList<>(); + List currentGroup = new ArrayList<>(); + int totalSizeSoFar = 0; + for (FileSlice currentSlice : fileSlices) { + // assume each filegroup size is ~= parquet.max.file.size + totalSizeSoFar += currentSlice.getBaseFile().isPresent() ? currentSlice.getBaseFile().get().getFileSize() : getWriteConfig().getParquetMaxFileSize(); + // check if max size is reached and create new group, if needed. + if (totalSizeSoFar >= getWriteConfig().getClusteringMaxBytesInGroup() && !currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, getNumberOfOutputFileGroups(totalSizeSoFar, getWriteConfig().getClusteringTargetFileMaxBytes()))); + currentGroup = new ArrayList<>(); + totalSizeSoFar = 0; + } + currentGroup.add(currentSlice); + } + if (!currentGroup.isEmpty()) { + fileSliceGroups.add(Pair.of(currentGroup, getNumberOfOutputFileGroups(totalSizeSoFar, getWriteConfig().getClusteringTargetFileMaxBytes()))); + } + + return fileSliceGroups.stream().map(fileSliceGroup -> HoodieClusteringGroup.newBuilder() + .setSlices(getFileSliceInfo(fileSliceGroup.getLeft())) + .setNumOutputFileGroups(fileSliceGroup.getRight()) + .setMetrics(buildMetrics(fileSliceGroup.getLeft())) + .build()); + } + + @Override + protected Map getStrategyParams() { + Map params = new HashMap<>(); + if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) { + params.put(CLUSTERING_SORT_COLUMNS_PROPERTY, getWriteConfig().getClusteringSortColumns()); + } + return params; + } + + @Override + protected List filterPartitionPaths(List partitionPaths) { + int targetPartitionsForClustering = getWriteConfig().getTargetPartitionsForClustering(); + return partitionPaths.stream() + .sorted(Comparator.reverseOrder()) + .limit(targetPartitionsForClustering > 0 ? targetPartitionsForClustering : partitionPaths.size()) + .collect(Collectors.toList()); + } + + @Override + protected Stream getFileSlicesEligibleForClustering(final String partition) { + return super.getFileSlicesEligibleForClustering(partition) + // Only files that have basefile size smaller than small file size are eligible. + .filter(slice -> slice.getBaseFile().map(HoodieBaseFile::getFileSize).orElse(0L) < getWriteConfig().getClusteringSmallFileLimit()); + } + + private int getNumberOfOutputFileGroups(long groupSize, long targetFileSize) { + return (int) Math.ceil(groupSize / (double) targetFileSize); + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java new file mode 100644 index 0000000000000..cfbc2ec22d319 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.run.strategy; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; +import org.apache.hudi.table.HoodieSparkMergeOnReadTable; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY; + +/** + * Clustering Strategy based on following. + * 1) Spark execution engine. + * 2) Uses bulk_insert to write data into new files. + */ +public class SparkSortAndSizeExecutionStrategy> + extends ClusteringExecutionStrategy>, JavaRDD, JavaRDD> { + private static final Logger LOG = LogManager.getLogger(SparkSortAndSizeExecutionStrategy.class); + + public SparkSortAndSizeExecutionStrategy(HoodieSparkCopyOnWriteTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + public SparkSortAndSizeExecutionStrategy(HoodieSparkMergeOnReadTable table, + HoodieSparkEngineContext engineContext, + HoodieWriteConfig writeConfig) { + super(table, engineContext, writeConfig); + } + + @Override + public JavaRDD performClustering(final JavaRDD> inputRecords, final int numOutputGroups, + final String instantTime, final Map strategyParams, final Schema schema) { + Properties props = getWriteConfig().getProps(); + props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM, String.valueOf(numOutputGroups)); + // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files. + props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, Boolean.FALSE.toString()); + HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build(); + return (JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + false, getPartitioner(strategyParams, schema), true, numOutputGroups); + } + + /** + * Create BulkInsertPartitioner based on strategy params. + */ + protected Option> getPartitioner(Map strategyParams, Schema schema) { + if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY)) { + return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY).split(","), + HoodieAvroUtils.addMetadataFields(schema))); + } else { + return Option.empty(); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java new file mode 100644 index 0000000000000..134e490246680 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.clustering.update.strategy; + +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieClusteringUpdateException; +import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.HashSet; +import java.util.List; + +/** + * Update strategy based on following. + * if some file group have update record, throw exception + */ +public class SparkRejectUpdateStrategy> extends UpdateStrategy>> { + private static final Logger LOG = LogManager.getLogger(SparkRejectUpdateStrategy.class); + + public SparkRejectUpdateStrategy(HoodieSparkEngineContext engineContext, HashSet fileGroupsInPendingClustering) { + super(engineContext, fileGroupsInPendingClustering); + } + + private List getGroupIdsWithUpdate(JavaRDD> inputRecords) { + List fileGroupIdsWithUpdates = inputRecords + .filter(record -> record.getCurrentLocation() != null) + .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collect(); + return fileGroupIdsWithUpdates; + } + + @Override + public JavaRDD> handleUpdate(JavaRDD> taggedRecordsRDD) { + List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); + fileGroupIdsWithRecordUpdate.forEach(fileGroupIdWithRecordUpdate -> { + if (fileGroupsInPendingClustering.contains(fileGroupIdWithRecordUpdate)) { + String msg = String.format("Not allowed to update the clustering file group %s. " + + "For pending clustering operations, we are not going to support update for now.", + fileGroupIdWithRecordUpdate.toString()); + LOG.error(msg); + throw new HoodieClusteringUpdateException(msg); + } + }); + return taggedRecordsRDD; + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index ec90ef88ed86f..27b5482b2c871 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -19,6 +19,7 @@ package org.apache.hudi.execution; import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; @@ -34,14 +35,18 @@ public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { + private boolean useWriterSchema; + public SparkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable, String idPrefix, - TaskContextSupplier taskContextSupplier) { + TaskContextSupplier taskContextSupplier, + boolean useWriterSchema) { super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); + this.useWriterSchema = useWriterSchema; } public SparkLazyInsertIterable(Iterator> recordItr, @@ -53,6 +58,7 @@ public SparkLazyInsertIterable(Iterator> recordItr, TaskContextSupplier taskContextSupplier, WriteHandleFactory writeHandleFactory) { super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory); + this.useWriterSchema = false; } @Override @@ -61,7 +67,10 @@ protected List computeNext() { BoundedInMemoryExecutor, HoodieInsertValueGenResult, List> bufferedIteratorExecutor = null; try { - final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); + if (useWriterSchema) { + schema = HoodieAvroUtils.addMetadataFields(schema); + } bufferedIteratorExecutor = new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema)); final List result = bufferedIteratorExecutor.execute(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java index db73a9c3e7e70..96da3969eecb2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java @@ -41,20 +41,22 @@ public class BulkInsertMapFunction private HoodieWriteConfig config; private HoodieTable hoodieTable; private List fileIDPrefixes; + private boolean useWriterSchema; public BulkInsertMapFunction(String instantTime, boolean areRecordsSorted, HoodieWriteConfig config, HoodieTable hoodieTable, - List fileIDPrefixes) { + List fileIDPrefixes, boolean useWriterSchema) { this.instantTime = instantTime; this.areRecordsSorted = areRecordsSorted; this.config = config; this.hoodieTable = hoodieTable; this.fileIDPrefixes = fileIDPrefixes; + this.useWriterSchema = useWriterSchema; } @Override public Iterator> call(Integer partition, Iterator> recordItr) { return new SparkLazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable, - fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier()); + fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier(), useWriterSchema); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java new file mode 100644 index 0000000000000..65c20004c81ed --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.execution.bulkinsert; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.BulkInsertPartitioner; +import org.apache.spark.api.java.JavaRDD; + +import java.io.IOException; + +/** + * A partitioner that does sorting based on specified column values for each RDD partition. + * + * @param HoodieRecordPayload type + */ +public class RDDCustomColumnsSortPartitioner + implements BulkInsertPartitioner>> { + + private final String[] sortColumnNames; + private final SerializableSchema serializableSchema; + + public RDDCustomColumnsSortPartitioner(String[] columnNames, Schema schema) { + this.sortColumnNames = columnNames; + this.serializableSchema = new SerializableSchema(schema); + } + + @Override + public JavaRDD> repartitionRecords(JavaRDD> records, + int outputSparkPartitions) { + final String[] sortColumns = this.sortColumnNames; + final SerializableSchema schema = this.serializableSchema; + return records.sortBy(record -> getRecordSortColumnValues(record, sortColumns, schema), + true, outputSparkPartitions); + } + + @Override + public boolean arePartitionRecordsSorted() { + return true; + } + + private static String getRecordSortColumnValues(HoodieRecord record, + String[] sortColumns, + SerializableSchema schema) { + try { + GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema.get()).get(); + StringBuilder sb = new StringBuilder(); + for (String col : sortColumns) { + sb.append(genericRecord.get(col)); + } + + return sb.toString(); + } catch (IOException e) { + throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 99a8f1f3c10c3..357b5cea888bd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -19,6 +19,7 @@ package org.apache.hudi.table; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -44,12 +45,15 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; import org.apache.hudi.table.action.clean.SparkCleanActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; -import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor; +import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; +import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkDeletePartitionCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; +import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkMergeHelper; import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor; @@ -57,7 +61,6 @@ import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor; import org.apache.hudi.table.action.savepoint.SavepointActionExecutor; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -106,6 +109,11 @@ public HoodieWriteMetadata> delete(HoodieEngineContext cont return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); } + @Override + public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + return new SparkDeletePartitionCommitActionExecutor(context, config, this, instantTime, partitions).execute(); + } + @Override public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, JavaRDD> preppedRecords) { @@ -145,6 +153,19 @@ public HoodieWriteMetadata> compact(HoodieEngineContext con throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } + @Override + public Option scheduleClustering(HoodieEngineContext context, + String instantTime, + Option> extraMetadata) { + return new SparkClusteringPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute(); + } + + @Override + public HoodieWriteMetadata> cluster(HoodieEngineContext context, + String clusteringInstantTime) { + return new SparkExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); + } + @Override public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java new file mode 100644 index 0000000000000..78510ba1c73f9 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkClusteringPlanActionExecutor.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster; + +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.cluster.strategy.ClusteringPlanStrategy; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; + +@SuppressWarnings("checkstyle:LineLength") +public class SparkClusteringPlanActionExecutor extends + BaseClusteringPlanActionExecutor>, JavaRDD, JavaRDD> { + + private static final Logger LOG = LogManager.getLogger(SparkClusteringPlanActionExecutor.class); + + public SparkClusteringPlanActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, JavaRDD, JavaRDD> table, + String instantTime, + Option> extraMetadata) { + super(context, config, table, instantTime, extraMetadata); + } + + @Override + protected Option createClusteringPlan() { + LOG.info("Checking if clustering needs to be run on " + config.getBasePath()); + Option lastClusteringInstant = table.getActiveTimeline().getCompletedReplaceTimeline().lastInstant(); + + int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) + .countInstants(); + if (config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { + LOG.info("Not scheduling clustering as only " + commitsSinceLastClustering + + " commits was found since last clustering " + lastClusteringInstant + ". Waiting for " + + config.getInlineClusterMaxCommits()); + return Option.empty(); + } + + LOG.info("Generating clustering plan for table " + config.getBasePath()); + ClusteringPlanStrategy strategy = (ClusteringPlanStrategy) + ReflectionUtils.loadClass(config.getClusteringPlanStrategyClass(), table, context, config); + return strategy.generateClusteringPlan(); + } + +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java new file mode 100644 index 0000000000000..a8044edcfcb20 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.cluster; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieClusteringGroup; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.SparkTaskContextSupplier; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.apache.hudi.common.model.ClusteringOperation; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.log.HoodieFileSliceReader; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieClusteringException; +import org.apache.hudi.io.IOUtils; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +public class SparkExecuteClusteringCommitActionExecutor> + extends BaseSparkCommitActionExecutor { + + private static final Logger LOG = LogManager.getLogger(SparkExecuteClusteringCommitActionExecutor.class); + private final HoodieClusteringPlan clusteringPlan; + + public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime) { + super(context, config, table, instantTime, WriteOperationType.CLUSTER); + this.clusteringPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), HoodieTimeline.getReplaceCommitRequestedInstant(instantTime)) + .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException("Unable to read clustering plan for instant: " + instantTime)); + } + + @Override + public HoodieWriteMetadata> execute() { + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime); + // Mark instant as clustering inflight + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + table.getMetaClient().reloadActiveTimeline(); + + JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(context); + // execute clustering for each group async and collect WriteStatus + JavaRDD writeStatusRDD = clusteringPlan.getInputGroups().stream() + .map(inputGroup -> runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams())) + .map(CompletableFuture::join) + .reduce((rdd1, rdd2) -> rdd1.union(rdd2)).orElse(engineContext.emptyRDD()); + if (writeStatusRDD.isEmpty()) { + throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + instantTime + " #groups: " + clusteringPlan.getInputGroups().size()); + } + + HoodieWriteMetadata> writeMetadata = buildWriteMetadata(writeStatusRDD); + updateIndexAndCommitIfNeeded(writeStatusRDD, writeMetadata); + if (!writeMetadata.getCommitMetadata().isPresent()) { + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStatusRDD.map(WriteStatus::getStat).collect(), writeMetadata.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + writeMetadata.setCommitMetadata(Option.of(commitMetadata)); + } + return writeMetadata; + } + + /** + * Submit job to execute clustering for the group. + */ + private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams) { + CompletableFuture> writeStatusesFuture = CompletableFuture.supplyAsync(() -> { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + JavaRDD> inputRecords = readRecordsForGroup(jsc, clusteringGroup); + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + return ((ClusteringExecutionStrategy>, JavaRDD, JavaRDD>) + ReflectionUtils.loadClass(config.getClusteringExecutionStrategyClass(), table, context, config)) + .performClustering(inputRecords, clusteringGroup.getNumOutputFileGroups(), instantTime, strategyParams, readerSchema); + }); + + return writeStatusesFuture; + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + @Override + protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { + return ClusteringUtils.getFileGroupsFromClusteringPlan(clusteringPlan).collect( + Collectors.groupingBy(fg -> fg.getPartitionPath(), Collectors.mapping(fg -> fg.getFileId(), Collectors.toList()))); + } + + /** + * Get RDD of all records for the group. This includes all records from file slice (Apply updates from log files, if any). + */ + private JavaRDD> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup) { + List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); + boolean hasLogFiles = clusteringOps.stream().filter(op -> op.getDeltaFilePaths().size() > 0).findAny().isPresent(); + if (hasLogFiles) { + // if there are log files, we read all records into memory for a file group and apply updates. + return readRecordsForGroupWithLogs(jsc, clusteringOps); + } else { + // We want to optimize reading records for case there are no log files. + return readRecordsForGroupBaseFiles(jsc, clusteringOps); + } + } + + /** + * Read records from baseFiles, apply updates and convert to RDD. + */ + private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext jsc, + List clusteringOps) { + return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + List>> recordIterators = new ArrayList<>(); + clusteringOpsPartition.forEachRemaining(clusteringOp -> { + long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps()); + LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction); + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(table.getMetaClient().getFs(), + table.getMetaClient().getBasePath(), clusteringOp.getDeltaFilePaths(), readerSchema, instantTime, + maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(), + config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(), + config.getSpillableMapBasePath()); + + recordIterators.add(HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema, + table.getMetaClient().getTableConfig().getPayloadClass())); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + + return new ConcatenatingIterator<>(recordIterators); + }); + } + + /** + * Read records from baseFiles and convert to RDD. + */ + private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext jsc, + List clusteringOps) { + return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + List> iteratorsForPartition = new ArrayList<>(); + clusteringOpsPartition.forEachRemaining(clusteringOp -> { + try { + Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())); + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())); + iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema)); + } catch (IOException e) { + throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath() + + " and " + clusteringOp.getDeltaFilePaths(), e); + } + }); + + return new ConcatenatingIterator<>(iteratorsForPartition); + }).map(this::transform); + } + + /** + * Transform IndexedRecord into HoodieRecord. + */ + private HoodieRecord transform(IndexedRecord indexedRecord) { + GenericRecord record = (GenericRecord) indexedRecord; + String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieKey hoodieKey = new HoodieKey(key, partition); + + HoodieRecordPayload avroPayload = ReflectionUtils.loadPayload(table.getMetaClient().getTableConfig().getPayloadClass(), + new Object[] {Option.of(record)}, Option.class); + HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload); + return hoodieRecord; + } + + private HoodieWriteMetadata> buildWriteMetadata(JavaRDD writeStatusJavaRDD) { + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(writeStatusJavaRDD)); + result.setWriteStatuses(writeStatusJavaRDD); + result.setWriteStats(writeStatusJavaRDD.map(WriteStatus::getStat).collect()); + result.setCommitMetadata(Option.empty()); + result.setCommitted(false); + return result; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index 73be8d4127413..1fd5dad688553 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 @@ -28,11 +28,13 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -46,6 +48,7 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; @@ -59,11 +62,13 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; +import java.util.stream.Collectors; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Set; import java.util.Map; public abstract class BaseSparkCommitActionExecutor extends @@ -88,6 +93,18 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, super(context, config, table, instantTime, operationType, extraMetadata); } + private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { + if (config.isClusteringEnabled()) { + Set fileGroupsInPendingClustering = + table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); + UpdateStrategy updateStrategy = (UpdateStrategy)ReflectionUtils + .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); + return (JavaRDD>)updateStrategy.handleUpdate(inputRecordsRDD); + } else { + return inputRecordsRDD; + } + } + @Override public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); @@ -107,9 +124,12 @@ public HoodieWriteMetadata> execute(JavaRDD saveWorkloadProfileMetadataToInflight(profile, instantTime); } + // handle records update with clustering + JavaRDD> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD); + // partition using the insert partitioner final Partitioner partitioner = getPartitioner(profile); - JavaRDD> partitionedRecords = partition(inputRecordsRDD, partitioner); + JavaRDD> partitionedRecords = partition(inputRecordsRDDWithClusteringUpdate, partitioner); JavaRDD writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> { if (WriteOperationType.isChangingRecords(operationType)) { return handleUpsertPartition(instantTime, partition, recordItr, partitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java index fb8b5f9cd3e05..2b00d47b01564 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java @@ -18,8 +18,6 @@ package org.apache.hudi.table.action.commit; -import java.util.Map; - import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.HoodieRecord; @@ -28,12 +26,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieInsertException; -import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.BulkInsertPartitioner; - +import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.spark.api.java.JavaRDD; +import java.util.Map; + public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { private final JavaRDD> inputRecordsRDD; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 9ccd66b2cd265..66fd68e15496c 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -31,7 +31,6 @@ import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; - import org.apache.spark.api.java.JavaRDD; import java.util.List; @@ -59,25 +58,45 @@ public static SparkBulkInsertHelper newInstance() { } @Override - public HoodieWriteMetadata> bulkInsert(JavaRDD> inputRecords, - String instantTime, - HoodieTable>, JavaRDD, JavaRDD> table, - HoodieWriteConfig config, - BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, - boolean performDedupe, - Option> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsert(final JavaRDD> inputRecords, + final String instantTime, + final HoodieTable>, JavaRDD, JavaRDD> table, + final HoodieWriteConfig config, + final BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, + final boolean performDedupe, + final Option> userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); + //transition bulk_insert state to inflight + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, + table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), + config.shouldAllowMultiWriteOnSameInstant()); + // write new files + JavaRDD writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism()); + //update index + ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); + return result; + } + + @Override + public JavaRDD bulkInsert(JavaRDD> inputRecords, + String instantTime, + HoodieTable>, JavaRDD, JavaRDD> table, + HoodieWriteConfig config, + boolean performDedupe, + Option> userDefinedBulkInsertPartitioner, + boolean useWriterSchema, + int parallelism) { + // De-dupe/merge if needed JavaRDD> dedupedRecords = inputRecords; if (performDedupe) { dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, - config.getBulkInsertShuffleParallelism(), table); + parallelism, table); } final JavaRDD> repartitionedRecords; - final int parallelism = config.getBulkInsertShuffleParallelism(); BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); @@ -87,16 +106,11 @@ public HoodieWriteMetadata> bulkInsert(JavaRDD fileIDPrefixes = IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList()); - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, - table.getMetaClient().getCommitActionType(), instantTime), Option.empty(), - config.shouldAllowMultiWriteOnSameInstant()); - JavaRDD writeStatusRDD = repartitionedRecords .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, - partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true) + partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, useWriterSchema), true) .flatMap(List::iterator); - ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatusRDD, result); - return result; + return writeStatusRDD; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java new file mode 100644 index 0000000000000..ea1ef51a4d40a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.WorkloadStat; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SparkDeletePartitionCommitActionExecutor> + extends SparkInsertOverwriteCommitActionExecutor { + + private List partitions; + public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, HoodieTable table, + String instantTime, List partitions) { + super(context, config, table, instantTime,null, WriteOperationType.DELETE_PARTITION); + this.partitions = partitions; + } + + @Override + public HoodieWriteMetadata> execute() { + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + HoodieTimer timer = new HoodieTimer().startTimer(); + Map> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct() + .mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + HoodieWriteMetadata result = new HoodieWriteMetadata(); + result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); + + result.setWriteStatuses(jsc.emptyRDD()); + this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); + this.commitOnAutoCommit(result); + return result; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 1e3822016a765..c5d3c7684c860 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -77,7 +77,7 @@ protected Map> getPartitionToReplacedFileIds(JavaRDD(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); } - private List getAllExistingFileIds(String partitionPath) { + protected List getAllExistingFileIds(String partitionPath) { // because new commit is not complete. it is safe to mark all existing file Ids as old files return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index e349657b7e44e..c0145158f3b71 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -36,7 +36,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; public class SparkInsertOverwriteTableCommitActionExecutor> extends SparkInsertOverwriteCommitActionExecutor { @@ -47,11 +46,6 @@ public SparkInsertOverwriteTableCommitActionExecutor(HoodieEngineContext context super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE); } - protected List getAllExistingFileIds(String partitionPath) { - return table.getSliceView().getLatestFileSlices(partitionPath) - .map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); - } - @Override protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { Map> partitionToExistingFileIds = new HashMap<>(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index b28c89a536469..ffcc1ed3ce205 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 @@ -129,6 +129,34 @@ private int addUpdateBucket(String partitionPath, String fileIdHint) { return bucket; } + /** + * Get the in pending clustering fileId for each partition path. + * @return partition path to pending clustering file groups id + */ + private Map> getPartitionPathToPendingClusteringFileGroupsId() { + Map> partitionPathToInPendingClusteringFileId = + table.getFileSystemView().getFileGroupsInPendingClustering() + .map(fileGroupIdAndInstantPair -> + Pair.of(fileGroupIdAndInstantPair.getKey().getPartitionPath(), fileGroupIdAndInstantPair.getKey().getFileId())) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toSet()))); + return partitionPathToInPendingClusteringFileId; + } + + /** + * Exclude small file handling for clustering since update path is not supported. + * @param pendingClusteringFileGroupsId pending clustering file groups id of partition + * @param smallFiles small files of partition + * @return smallFiles not in clustering + */ + private List filterSmallFilesInClustering(final Set pendingClusteringFileGroupsId, final List smallFiles) { + if (this.config.isClusteringEnabled()) { + return smallFiles.stream() + .filter(smallFile -> !pendingClusteringFileGroupsId.contains(smallFile.location.getFileId())).collect(Collectors.toList()); + } else { + return smallFiles; + } + } + private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) { // for new inserts, compute buckets depending on how many records we have for each partition Set partitionPaths = profile.getPartitionPaths(); @@ -140,11 +168,16 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) Map> partitionSmallFilesMap = getSmallFilesForPartitions(new ArrayList(partitionPaths), context); + Map> partitionPathToPendingClusteringFileGroupsId = getPartitionPathToPendingClusteringFileGroupsId(); + for (String partitionPath : partitionPaths) { WorkloadStat pStat = profile.getWorkloadStat(partitionPath); if (pStat.getNumInserts() > 0) { - List smallFiles = partitionSmallFilesMap.get(partitionPath); + List smallFiles = + filterSmallFilesInClustering(partitionPathToPendingClusteringFileGroupsId.getOrDefault(partitionPath, Collections.emptySet()), + partitionSmallFilesMap.get(partitionPath)); + this.smallFiles.addAll(smallFiles); LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles); @@ -185,7 +218,11 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) + ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket); for (int b = 0; b < insertBuckets; b++) { bucketNumbers.add(totalBuckets); - recordsPerBucket.add(totalUnassignedInserts / insertBuckets); + if (b < insertBuckets - 1) { + recordsPerBucket.add(insertRecordsPerBucket); + } else { + recordsPerBucket.add(totalUnassignedInserts - (insertBuckets - 1) * insertRecordsPerBucket); + } BucketInfo bucketInfo = new BucketInfo(); bucketInfo.bucketType = BucketType.INSERT; bucketInfo.partitionPath = partitionPath; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 65cefc9b9923c..96d52a1605c62 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -18,6 +18,9 @@ package org.apache.hudi.table.action.compact; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.avro.model.HoodieCompactionPlan; @@ -49,10 +52,6 @@ import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -179,7 +178,8 @@ private List compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteT @Override public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, HoodieTable>, JavaRDD, JavaRDD> hoodieTable, - HoodieWriteConfig config, String compactionCommitTime, Set fgIdsInPendingCompactions) + HoodieWriteConfig config, String compactionCommitTime, + Set fgIdsInPendingCompactionAndClustering) throws IOException { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); totalLogFiles = new LongAccumulator(); @@ -213,7 +213,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, List operations = context.flatMap(partitionPaths, partitionPath -> { return fileSystemView .getLatestFileSlices(partitionPath) - .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())) + .filter(slice -> !fgIdsInPendingCompactionAndClustering.contains(slice.getFileGroupId())) .map(s -> { List logFiles = s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); @@ -224,7 +224,7 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, // into meta files. Option dataFile = s.getBaseFile(); return new CompactionOperation(dataFile, partitionPath, logFiles, - config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); + config.getCompactionStrategy().captureMetrics(config, s)); }) .filter(c -> !c.getDeltaFileNames().isEmpty()); }, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList()); @@ -239,9 +239,9 @@ public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList())); ValidationUtils.checkArgument( compactionPlan.getOperations().stream().noneMatch( - op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), + op -> fgIdsInPendingCompactionAndClustering.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))), "Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. " - + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions + + "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactionAndClustering + ", Selected workload :" + compactionPlan); if (compactionPlan.getOperations().isEmpty()) { LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java index c5f6c1692c91e..96d76d356092a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java @@ -21,22 +21,24 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCompactionException; import org.apache.hudi.table.HoodieTable; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; import java.io.IOException; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; @SuppressWarnings("checkstyle:LineLength") @@ -75,10 +77,13 @@ protected HoodieCompactionPlan scheduleCompaction() { LOG.info("Generating compaction plan for merge on read table " + config.getBasePath()); HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor(); try { - return compactor.generateCompactionPlan(context, table, config, instantTime, - ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations() - .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) - .collect(Collectors.toSet())); + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + Set fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations() + .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId()) + .collect(Collectors.toSet()); + // exclude files in pending clustering from compaction. + fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet())); + return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering); } catch (IOException e) { throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index d278b08f3fc14..e86cb2d4f9a38 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -21,27 +21,35 @@ import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; @@ -50,12 +58,15 @@ import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.MarkerFiles; +import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkWriteHelper; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -75,6 +86,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -82,6 +94,8 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.Properties; import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; import static org.apache.hudi.common.testutils.FileCreateUtils.getBaseFileCountsForPaths; @@ -93,6 +107,8 @@ import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys; import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet; import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet; +import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY; +import static org.apache.hudi.config.HoodieClusteringConfig.DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -106,6 +122,12 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase { private static final Logger LOG = LogManager.getLogger(TestHoodieClientOnCopyOnWriteStorage.class); + private static final Map STRATEGY_PARAMS = new HashMap() { + { + put("sortColumn", "record_key"); + } + }; + private HoodieTestTable testTable; @BeforeEach @@ -677,6 +699,70 @@ private void assertActualAndExpectedPartitionPathRecordKeyMatches(Set, List> insertBatchRecords(SparkRDDWriteClient client, String commitTime, + Integer recordNum, int expectStatueSize) { + client.startCommitWithTime(commitTime); + List inserts1 = dataGen.generateInserts(commitTime, recordNum); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 1); + List statuses = client.upsert(insertRecordsRDD1, commitTime).collect(); + assertNoWriteErrors(statuses); + assertEquals(expectStatueSize, statuses.size(), "check expect statue size."); + return Pair.of(statuses, inserts1); + } + + @Test + public void testUpdateRejectForClustering() throws IOException { + final String testPartitionPath = "2016/09/26"; + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + Properties props = new Properties(); + props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, "true"); + HoodieWriteConfig config = getSmallInsertWriteConfig(100, + TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient); + + //1. insert to generate 2 file group + String commitTime1 = "001"; + Pair, List> upsertResult = insertBatchRecords(client, commitTime1, 600, 2); + List inserts1 = upsertResult.getValue(); + List fileGroupIds1 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(2, fileGroupIds1.size()); + + // 2. generate clustering plan for fileGroupIds1 file groups + String commitTime2 = "002"; + List> firstInsertFileSlicesList = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getAllFileSlices().collect(Collectors.toList())).collect(Collectors.toList()); + List[] fileSlices = (List[])firstInsertFileSlicesList.toArray(new List[firstInsertFileSlicesList.size()]); + createRequestedReplaceInstant(this.metaClient, commitTime2, fileSlices); + + // 3. insert one record with no updating reject exception, and not merge the small file, just generate a new file group + String commitTime3 = "003"; + insertBatchRecords(client, commitTime3, 1, 1).getKey(); + List fileGroupIds2 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(3, fileGroupIds2.size()); + + // 4. update one record for the clustering two file groups, throw reject update exception + String commitTime4 = "004"; + client.startCommitWithTime(commitTime4); + List insertsAndUpdates3 = new ArrayList<>(); + insertsAndUpdates3.addAll(dataGen.generateUpdates(commitTime4, inserts1)); + String assertMsg = String.format("Not allowed to update the clustering files in partition: %s " + + "For pending clustering operations, we are not going to support update for now.", testPartitionPath); + assertThrows(HoodieUpsertException.class, () -> { + writeClient.upsert(jsc.parallelize(insertsAndUpdates3, 1), commitTime3).collect(); }, assertMsg); + + // 5. insert one record with no updating reject exception, will merge the small file + String commitTime5 = "005"; + List statuses = insertBatchRecords(client, commitTime5, 1, 1).getKey(); + fileGroupIds2.removeAll(fileGroupIds1); + assertEquals(fileGroupIds2.get(0), statuses.get(0).getFileId()); + List firstInsertFileGroupIds4 = table.getFileSystemView().getAllFileGroups(testPartitionPath) + .map(fileGroup -> fileGroup.getFileGroupId().getFileId()).collect(Collectors.toList()); + assertEquals(3, firstInsertFileGroupIds4.size()); + } + /** * Test scenario of new file-group getting added during upsert(). */ @@ -938,6 +1024,65 @@ public void testDeletesWithDeleteApi() throws Exception { testDeletes(client, updateBatch3.getRight(), 10, file1, "007", 140, keysSoFar); } + @Test + public void testSimpleClustering() throws Exception { + // setup clustering config + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + testClustering(clusteringConfig); + } + + @Test + public void testClusteringWithSortColumns() throws Exception { + // setup clustering config + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringSortColumns("_hoodie_record_key") + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + testClustering(clusteringConfig); + } + + private void testClustering(HoodieClusteringConfig clusteringConfig) throws Exception { + // create config to not update small files. + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false, 10); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + dataGen = new HoodieTestDataGenerator(); + String commitTime = "100"; + List records1 = dataGen.generateInserts(commitTime, 200); + List statuses1 = writeAndVerifyBatch(client, records1, commitTime); + Set fileIds1 = getFileGroupIdsFromWriteStatus(statuses1); + + commitTime = "200"; + List records2 = dataGen.generateInserts(commitTime, 200); + List statuses2 = writeAndVerifyBatch(client, records2, commitTime); + Set fileIds2 = getFileGroupIdsFromWriteStatus(statuses2); + //verify new files are created for 2nd write + Set fileIdIntersection = new HashSet<>(fileIds1); + fileIdIntersection.retainAll(fileIds2); + assertEquals(0, fileIdIntersection.size()); + + config = getConfigBuilder().withClusteringConfig(clusteringConfig).build(); + + // create client with new config. + client = getHoodieWriteClient(config, false); + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + HoodieWriteMetadata> clusterMetadata = client.cluster(clusteringCommitTime, true); + List allRecords = Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList()); + verifyRecordsWritten(clusteringCommitTime, allRecords, clusterMetadata.getWriteStatuses().collect()); + Set insertedFileIds = new HashSet<>(); + insertedFileIds.addAll(fileIds1); + insertedFileIds.addAll(fileIds2); + + Set replacedFileIds = new HashSet<>(); + clusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles -> + partitionFiles.getValue().stream().forEach(file -> + replacedFileIds.add(new HoodieFileGroupId(partitionFiles.getKey(), file)))); + assertEquals(insertedFileIds, replacedFileIds); + } + + private Set getFileGroupIdsFromWriteStatus(List statuses) { + return statuses.stream().map(s -> new HoodieFileGroupId(s.getPartitionPath(), s.getFileId())).collect(Collectors.toSet()); + } + /** * Test scenario of writing more file groups than existing number of file groups in partition. */ @@ -958,7 +1103,7 @@ public void testInsertOverwritePartitionHandlingWithFewerRecords() throws Except * Test scenario of writing similar number file groups in partition. */ @Test - public void testInsertOverwritePartitionHandlinWithSimilarNumberOfRecords() throws Exception { + public void testInsertOverwritePartitionHandlingWithSimilarNumberOfRecords() throws Exception { verifyInsertOverwritePartitionHandling(3000, 3000); } @@ -975,14 +1120,9 @@ private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); // Do Inserts - String commitTime1 = "001"; - client.startCommitWithTime(commitTime1); - List inserts1 = dataGen.generateInserts(commitTime1, batch1RecordsCount); - JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); - List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); - assertNoWriteErrors(statuses); - Set batch1Buckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); - verifyRecordsWritten(commitTime1, inserts1, statuses); + String commit1 = "001"; + List statuses = writeAndVerifyBatch(client, dataGen.generateInserts(commit1, batch1RecordsCount), commit1); + Set batch1Buckets = getFileIdsFromWriteStatus(statuses); // Do Insert Overwrite String commitTime2 = "002"; @@ -999,6 +1139,113 @@ private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int verifyRecordsWritten(commitTime2, inserts2, statuses); } + private Set getFileIdsFromWriteStatus(List statuses) { + return statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); + } + + /** + * Test scenario of writing fewer file groups for first partition than second an third partition. + */ + @Test + public void verifyDeletePartitionsHandlingWithFewerRecordsFirstPartition() throws Exception { + verifyDeletePartitionsHandling(1000, 3000, 3000); + } + + /** + * Test scenario of writing similar number file groups in partition. + */ + @Test + public void verifyDeletePartitionsHandlingWithSimilarNumberOfRecords() throws Exception { + verifyDeletePartitionsHandling(3000, 3000, 3000); + } + + /** + * Test scenario of writing more file groups for first partition than second an third partition. + */ + @Test + public void verifyDeletePartitionsHandlingHandlingWithFewerRecordsSecondThirdPartition() throws Exception { + verifyDeletePartitionsHandling(3000, 1000, 1000); + } + + private Set insertPartitionRecordsWithCommit(SparkRDDWriteClient client, int recordsCount, String commitTime1, String partitionPath) { + client.startCommitWithTime(commitTime1); + List inserts1 = dataGen.generateInsertsForPartition(commitTime1, recordsCount, partitionPath); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); + List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); + assertNoWriteErrors(statuses); + Set batchBuckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); + verifyRecordsWritten(commitTime1, inserts1, statuses); + return batchBuckets; + } + + private Set deletePartitionWithCommit(SparkRDDWriteClient client, String commitTime, List deletePartitionPath) { + client.startCommitWithTime(commitTime, HoodieTimeline.REPLACE_COMMIT_ACTION); + HoodieWriteResult writeResult = client.deletePartitions(deletePartitionPath, commitTime); + Set deletePartitionReplaceFileIds = + writeResult.getPartitionToReplaceFileIds().entrySet() + .stream().flatMap(entry -> entry.getValue().stream()).collect(Collectors.toSet()); + return deletePartitionReplaceFileIds; + } + + /** + * 1) Do write1 (upsert) with 'batch1RecordsCount' number of records for first partition. + * 2) Do write2 (upsert) with 'batch2RecordsCount' number of records for second partition. + * 3) Do write3 (upsert) with 'batch3RecordsCount' number of records for third partition. + * 4) delete first partition and check result. + * 5) delete second and third partition and check result. + * + */ + private void verifyDeletePartitionsHandling(int batch1RecordsCount, int batch2RecordsCount, int batch3RecordsCount) throws Exception { + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); + SparkRDDWriteClient client = getHoodieWriteClient(config, false); + dataGen = new HoodieTestDataGenerator(); + + // Do Inserts for DEFAULT_FIRST_PARTITION_PATH + String commitTime1 = "001"; + Set batch1Buckets = + this.insertPartitionRecordsWithCommit(client, batch1RecordsCount, commitTime1, DEFAULT_FIRST_PARTITION_PATH); + + // Do Inserts for DEFAULT_SECOND_PARTITION_PATH + String commitTime2 = "002"; + Set batch2Buckets = + this.insertPartitionRecordsWithCommit(client, batch2RecordsCount, commitTime2, DEFAULT_SECOND_PARTITION_PATH); + + // Do Inserts for DEFAULT_THIRD_PARTITION_PATH + String commitTime3 = "003"; + Set batch3Buckets = + this.insertPartitionRecordsWithCommit(client, batch3RecordsCount, commitTime3, DEFAULT_THIRD_PARTITION_PATH); + + // delete DEFAULT_FIRST_PARTITION_PATH + String commitTime4 = "004"; + Set deletePartitionReplaceFileIds1 = + deletePartitionWithCommit(client, commitTime4, Arrays.asList(DEFAULT_FIRST_PARTITION_PATH)); + assertEquals(batch1Buckets, deletePartitionReplaceFileIds1); + List baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_FIRST_PARTITION_PATH)); + assertEquals(0, baseFiles.size()); + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_SECOND_PARTITION_PATH)); + assertTrue(baseFiles.size() > 0); + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_THIRD_PARTITION_PATH)); + assertTrue(baseFiles.size() > 0); + + // delete DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH + String commitTime5 = "005"; + Set deletePartitionReplaceFileIds2 = + deletePartitionWithCommit(client, commitTime5, Arrays.asList(DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH)); + Set expectedFileId = new HashSet<>(); + expectedFileId.addAll(batch2Buckets); + expectedFileId.addAll(batch3Buckets); + assertEquals(expectedFileId, deletePartitionReplaceFileIds2); + + baseFiles = HoodieClientTestUtils.getLatestBaseFiles(basePath, fs, + String.format("%s/%s/*", basePath, DEFAULT_FIRST_PARTITION_PATH), + String.format("%s/%s/*", basePath, DEFAULT_SECOND_PARTITION_PATH), + String.format("%s/%s/*", basePath, DEFAULT_THIRD_PARTITION_PATH)); + assertEquals(0, baseFiles.size()); + } + /** * Verify data in parquet files matches expected records and commit time. */ @@ -1019,6 +1266,15 @@ private void verifyRecordsWritten(String commitTime, List expected } } + private List writeAndVerifyBatch(SparkRDDWriteClient client, List inserts, String commitTime) { + client.startCommitWithTime(commitTime); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts, 2); + List statuses = client.upsert(insertRecordsRDD1, commitTime).collect(); + assertNoWriteErrors(statuses); + verifyRecordsWritten(commitTime, inserts, statuses); + return statuses; + } + private Pair, List> testUpdates(String instantTime, SparkRDDWriteClient client, int sizeToInsertAndUpdate, int expectedTotalRecords) throws IOException { @@ -1386,16 +1642,45 @@ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) { * Build Hoodie Write Config for small data file sizes. */ private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema) { - HoodieWriteConfig.Builder builder = getConfigBuilder(useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA); + return getSmallInsertWriteConfig(insertSplitSize, useNullSchema, dataGen.getEstimatedFileSizeInBytes(150)); + } + + /** + * Build Hoodie Write Config for specified small file sizes. + */ + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, boolean useNullSchema, long smallFileSize) { + String schemaStr = useNullSchema ? NULL_SCHEMA : TRIP_EXAMPLE_SCHEMA; + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize); + } + + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize) { + return getSmallInsertWriteConfig(insertSplitSize, schemaStr, smallFileSize, new Properties()); + } + + private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize, String schemaStr, long smallFileSize, Properties props) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr); return builder .withCompactionConfig( HoodieCompactionConfig.newBuilder() - .compactionSmallFileSize(dataGen.getEstimatedFileSizeInBytes(150)) + .compactionSmallFileSize(smallFileSize) .insertSplitSize(insertSplitSize).build()) .withStorageConfig( HoodieStorageConfig.newBuilder() .hfileMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)) .parquetMaxFileSize(dataGen.getEstimatedFileSizeInBytes(200)).build()) + .withProps(props) .build(); } + + protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient metaClient, String clusterTime, List[] fileSlices) throws IOException { + HoodieClusteringPlan clusteringPlan = + ClusteringUtils.createClusteringPlan(DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS, STRATEGY_PARAMS, fileSlices, Collections.emptyMap()); + + HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime); + HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() + .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); + metaClient.getActiveTimeline().saveToPendingReplaceCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); + return clusteringInstant; + } + } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java index 834229b683f1e..81effaa2a7999 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java @@ -18,17 +18,23 @@ package org.apache.hudi.execution.bulkinsert; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.testutils.HoodieClientTestBase; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -40,6 +46,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase { + private static final Comparator> KEY_COMPARATOR = + Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())); public static JavaRDD generateTestRecordsForBulkInsert(JavaSparkContext jsc) { HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); @@ -69,9 +77,10 @@ private static Stream configParams() { return Stream.of(data).map(Arguments::of); } - private void verifyRecordAscendingOrder(List records) { - List expectedRecords = new ArrayList<>(records); - Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey()))); + private void verifyRecordAscendingOrder(List> records, + Option>> comparator) { + List> expectedRecords = new ArrayList<>(records); + Collections.sort(expectedRecords, comparator.orElse(KEY_COMPARATOR)); assertEquals(expectedRecords, records); } @@ -79,19 +88,28 @@ private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner JavaRDD records, boolean isGloballySorted, boolean isLocallySorted, Map expectedPartitionNumRecords) { + testBulkInsertInternalPartitioner(partitioner, records, isGloballySorted, isLocallySorted, expectedPartitionNumRecords, Option.empty()); + } + + private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner, + JavaRDD records, + boolean isGloballySorted, boolean isLocallySorted, + Map expectedPartitionNumRecords, + Option>> comparator) { int numPartitions = 2; - JavaRDD actualRecords = (JavaRDD) partitioner.repartitionRecords(records, numPartitions); + JavaRDD> actualRecords = + (JavaRDD>) partitioner.repartitionRecords(records, numPartitions); assertEquals(numPartitions, actualRecords.getNumPartitions()); - List collectedActualRecords = actualRecords.collect(); + List> collectedActualRecords = actualRecords.collect(); if (isGloballySorted) { // Verify global order - verifyRecordAscendingOrder(collectedActualRecords); + verifyRecordAscendingOrder(collectedActualRecords, comparator); } else if (isLocallySorted) { // Verify local order actualRecords.mapPartitions(partition -> { - List partitionRecords = new ArrayList<>(); + List> partitionRecords = new ArrayList<>(); partition.forEachRemaining(partitionRecords::add); - verifyRecordAscendingOrder(partitionRecords); + verifyRecordAscendingOrder(partitionRecords, comparator); return Collections.emptyList().iterator(); }).collect(); } @@ -118,4 +136,35 @@ public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode, testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode), records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2)); } + + @Test + public void testCustomColumnSortPartitioner() throws Exception { + String[] sortColumns = new String[] {"rider"}; + Comparator> columnComparator = getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns); + + JavaRDD records1 = generateTestRecordsForBulkInsert(jsc); + JavaRDD records2 = generateTripleTestRecordsForBulkInsert(jsc); + testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA), + records1, true, true, generateExpectedPartitionNumRecords(records1), Option.of(columnComparator)); + testBulkInsertInternalPartitioner(new RDDCustomColumnsSortPartitioner(sortColumns, HoodieTestDataGenerator.AVRO_SCHEMA), + records2, true, true, generateExpectedPartitionNumRecords(records2), Option.of(columnComparator)); + } + + private Comparator> getCustomColumnComparator(Schema schema, String[] sortColumns) { + Comparator> comparator = Comparator.comparing(record -> { + try { + GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); + StringBuilder sb = new StringBuilder(); + for (String col : sortColumns) { + sb.append(genericRecord.get(col)); + } + + return sb.toString(); + } catch (IOException e) { + throw new HoodieIOException("unable to read value for " + sortColumns); + } + }); + + return comparator; + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java index 2eb672a00bd4c..fe040c9432000 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java @@ -18,8 +18,10 @@ package org.apache.hudi.index.hbase; +import avro.shaded.com.google.common.collect.Maps; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -119,6 +121,8 @@ public static void init() throws Exception { public void setUp() throws Exception { hadoopConf = jsc().hadoopConfiguration(); hadoopConf.addResource(utility.getConfiguration()); + // reInit the context here to keep the hadoopConf the same with that in this class + context = new HoodieSparkEngineContext(jsc()); metaClient = getHoodieMetaClient(hadoopConf, basePath()); dataGen = new HoodieTestDataGenerator(); } @@ -307,6 +311,125 @@ public void testSimpleTagLocationAndUpdateWithRollback() throws Exception { assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count()); } + /* + * Test case to verify that for taglocation entries present in HBase, if the corresponding commit instant is missing + * in timeline and the commit is not archived, taglocation would reset the current record location to null. + */ + @Test + public void testSimpleTagLocationWithInvalidCommit() throws Exception { + // Load to memory + HoodieWriteConfig config = getConfig(); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + String newCommitTime = writeClient.startCommit(); + // make a commit with 199 records + JavaRDD writeRecords = generateAndCommitRecords(writeClient, 199); + + // make a second commit with a single record + String invalidCommit = writeClient.startCommit(); + JavaRDD invalidWriteRecords = generateAndCommitRecords(writeClient, 1, invalidCommit); + + // verify location is tagged. + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD0 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + assert (javaRDD0.collect().size() == 1); // one record present + assert (javaRDD0.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 1); // it is tagged + assert (javaRDD0.collect().get(0).getCurrentLocation().getInstantTime().equals(invalidCommit)); + + // rollback the invalid commit, so that hbase will be left with a stale entry. + writeClient.rollback(invalidCommit); + + // Now tagLocation for the valid records, hbaseIndex should tag them + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords, context(), hoodieTable); + assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 199); + + // tagLocation for the invalid record - commit is not present in timeline due to rollback. + JavaRDD javaRDD2 = index.tagLocation(invalidWriteRecords, context(), hoodieTable); + assert (javaRDD2.collect().size() == 1); // one record present + assert (javaRDD2.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0); // it is not tagged + } + + /* + * Test case to verify that taglocation() uses the commit timeline to validate the commitTS stored in hbase. + * When CheckIfValidCommit() in HbaseIndex uses the incorrect timeline filtering, this test would fail. + */ + @Test + public void testEnsureTagLocationUsesCommitTimeline() throws Exception { + // Load to memory + HoodieWriteConfig config = getConfig(); + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + String commitTime1 = writeClient.startCommit(); + JavaRDD writeRecords1 = generateAndCommitRecords(writeClient, 20, commitTime1); + + // rollback the commit - leaves a clean file in timeline. + writeClient.rollback(commitTime1); + + // create a second commit with 20 records + metaClient = HoodieTableMetaClient.reload(metaClient); + generateAndCommitRecords(writeClient, 20); + + // Now tagLocation for the first set of rolledback records, hbaseIndex should tag them + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + assert (javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 20); + } + + private JavaRDD generateAndCommitRecords(SparkRDDWriteClient writeClient, int numRecs) throws Exception { + String commitTime = writeClient.startCommit(); + return generateAndCommitRecords(writeClient, numRecs, commitTime); + } + + private JavaRDD generateAndCommitRecords(SparkRDDWriteClient writeClient, + int numRecs, String commitTime) throws Exception { + // first batch of records + List records = dataGen.generateInserts(commitTime, numRecs); + JavaRDD writeRecords = jsc().parallelize(records, 1); + metaClient = HoodieTableMetaClient.reload(metaClient); + + // Insert records + JavaRDD writeStatues = writeClient.upsert(writeRecords, commitTime); + assertNoWriteErrors(writeStatues.collect()); + + // commit this upsert + writeClient.commit(commitTime, writeStatues); + + return writeRecords; + } + + // Verify hbase is tagging records belonging to an archived commit as valid. + @Test + public void testHbaseTagLocationForArchivedCommits() throws Exception { + // Load to memory + Map params = Maps.newHashMap(); + params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1"); + params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "3"); + params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2"); + HoodieWriteConfig config = getConfigBuilder(100, false).withProps(params).build(); + + SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config); + SparkRDDWriteClient writeClient = getHoodieWriteClient(config); + + // make first commit with 20 records + JavaRDD writeRecords1 = generateAndCommitRecords(writeClient, 20); + + // Make 3 additional commits, so that first commit is archived + for (int nCommit = 0; nCommit < 3; nCommit++) { + generateAndCommitRecords(writeClient, 20); + } + + // tagLocation for the first set of records (for the archived commit), hbaseIndex should tag them as valid + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient); + JavaRDD javaRDD1 = index.tagLocation(writeRecords1, context(), hoodieTable); + assertEquals(20, javaRDD1.filter(HoodieRecord::isCurrentLocationKnown).collect().size()); + } + @Test public void testTotalGetsBatching() throws Exception { HoodieWriteConfig config = getConfig(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 8b47fa3d49457..42584b12514b4 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -18,6 +18,11 @@ package org.apache.hudi.table; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -46,6 +51,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.Transformations; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieStorageConfig; @@ -66,12 +73,6 @@ import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; import org.apache.hudi.testutils.HoodieWriteableTestTable; import org.apache.hudi.testutils.MetadataMergeWriteStatus; - -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -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.BeforeEach; @@ -228,6 +229,81 @@ public void testSimpleInsertAndUpdateHFile() throws Exception { } } + @Test + public void testSimpleClusteringNoUpdates() throws Exception { + testClustering(false); + } + + @Test + public void testSimpleClusteringWithUpdates() throws Exception { + testClustering(true); + } + + private void testClustering(boolean doUpdates) throws Exception { + // set low compaction small File Size to generate more file groups. + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + HoodieWriteConfig cfg = getConfigBuilder(true, 10L, clusteringConfig).build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) { + + /** + * Write 1 (only inserts) + */ + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 400); + insertAndGetFilePaths(records.subList(0, 200), client, cfg, newCommitTime); + + /** + * Write 2 (more inserts to create new files) + */ + // we already set small file size to small number to force inserts to go into new file. + newCommitTime = "002"; + client.startCommitWithTime(newCommitTime); + insertAndGetFilePaths(records.subList(200, 400), client, cfg, newCommitTime); + + if (doUpdates) { + /** + * Write 3 (updates) + */ + newCommitTime = "003"; + client.startCommitWithTime(newCommitTime); + records = dataGen.generateUpdates(newCommitTime, 100); + updateAndGetFilePaths(records, client, cfg, newCommitTime); + } + + HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); + FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable); + // expect 2 base files for each partition + assertEquals(dataGen.getPartitionPaths().length * 2, allFiles.length); + + String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString(); + metaClient = HoodieTableMetaClient.reload(metaClient); + hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); + // verify all files are included in clustering plan. + assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count()); + + // Do the clustering and validate + client.cluster(clusteringCommitTime, true); + + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context, metaClient); + Stream dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths()) + .flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p)); + // verify there should be only one base file per partition after clustering. + assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count()); + + HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); + assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), + "Expecting a single commit."); + assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); + assertEquals(400, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"), + "Must contain 200 records"); + } + } + // test incremental read does not go past compaction instant for RO views // For RT views, incremental read can go past compaction @Test @@ -1469,17 +1545,27 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieI return getConfigBuilder(autoCommit, false, indexType); } + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, long compactionSmallFileSize, HoodieClusteringConfig clusteringConfig) { + return getConfigBuilder(autoCommit, false, IndexType.BLOOM, compactionSmallFileSize, clusteringConfig); + } + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean rollbackUsingMarkers, HoodieIndex.IndexType indexType) { + return getConfigBuilder(autoCommit, rollbackUsingMarkers, indexType, 1024 * 1024 * 1024L, HoodieClusteringConfig.newBuilder().build()); + } + + protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean rollbackUsingMarkers, HoodieIndex.IndexType indexType, + long compactionSmallFileSize, HoodieClusteringConfig clusteringConfig) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withDeleteParallelism(2) .withAutoCommit(autoCommit).withAssumeDatePartitioning(true) - .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) + .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(compactionSmallFileSize) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") .withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder() .withEnableBackupForRemoteFileSystemView(false).build()) .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build()) + .withClusteringConfig(clusteringConfig) .withRollbackUsingMarkers(rollbackUsingMarkers); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index c19427c7f809a..f40a97c0bbadc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -185,6 +185,21 @@ public void testUpsertPartitioner() throws Exception { assertEquals(2, insertBuckets.size(), "Total of 2 insert buckets"); } + @Test + public void testUpsertPartitionerWithRecordsPerBucket() throws Exception { + final String testPartitionPath = "2016/09/26"; + // Inserts + Updates... Check all updates go together & inserts subsplit + UpsertPartitioner partitioner = getUpsertPartitioner(0, 250, 100, 1024, testPartitionPath, false); + List insertBuckets = partitioner.getInsertBuckets(testPartitionPath); + int insertSplitSize = partitioner.config.getCopyOnWriteInsertSplitSize(); + int remainedInsertSize = 250 - 2 * insertSplitSize; + // will assigned 3 insertBuckets. 100, 100, 50 each + assertEquals(3, insertBuckets.size(), "Total of 3 insert buckets"); + assertEquals(0.4, insertBuckets.get(0).getLeft().weight, "insert " + insertSplitSize + " records"); + assertEquals(0.4, insertBuckets.get(1).getLeft().weight, "insert " + insertSplitSize + " records"); + assertEquals(0.2, insertBuckets.get(2).getLeft().weight, "insert " + remainedInsertSize + " records"); + } + @Test public void testPartitionWeight() throws Exception { final String testPartitionPath = "2016/09/26"; @@ -286,8 +301,8 @@ public void testUpsertPartitionerWithSmallInsertHandling() throws Exception { "Bucket 3 is INSERT"); assertEquals(4, insertBuckets.size(), "Total of 4 insert buckets"); - weights = new Double[] { 0.08, 0.31, 0.31, 0.31}; - cumulativeWeights = new Double[] { 0.08, 0.39, 0.69, 1.0}; + weights = new Double[] { 0.08, 0.42, 0.42, 0.08}; + cumulativeWeights = new Double[] { 0.08, 0.5, 0.92, 1.0}; assertInsertBuckets(weights, cumulativeWeights, insertBuckets); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java index d140b1183aacf..faf7e7d43c0a5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java @@ -20,14 +20,13 @@ import org.apache.hudi.avro.model.HoodieCompactionOperation; import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; - -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; import java.text.SimpleDateFormat; @@ -257,10 +256,13 @@ private List createCompactionOperations(HoodieWriteCo HoodieBaseFile df = TestHoodieBaseFile.newDataFile(k); String partitionPath = keyToPartitionMap.get(k); List logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()); + FileSlice slice = new FileSlice(new HoodieFileGroupId(partitionPath, df.getFileId()), df.getCommitTime()); + slice.setBaseFile(df); + logFiles.stream().forEach(f -> slice.addLogFile(f)); operations.add(new HoodieCompactionOperation(df.getCommitTime(), logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(), partitionPath, - config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles), + config.getCompactionStrategy().captureMetrics(config, slice), df.getBootstrapBaseFile().map(BaseFile::getPath).orElse(null)) ); }); @@ -303,10 +305,11 @@ public long getFileSize() { public static class TestHoodieLogFile extends HoodieLogFile { + private static int version = 0; private final long size; public TestHoodieLogFile(long size) { - super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1"); + super("/tmp/.ce481ee7-9e53-4a2e-999-f9e295fa79c0_20180919184844.log." + version++); this.size = size; } @@ -314,11 +317,6 @@ public static HoodieLogFile newLogFile(long size) { return new TestHoodieLogFile(size); } - @Override - public Path getPath() { - return new Path("/tmp/test-log"); - } - @Override public long getFileSize() { return size; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 307e0686756a8..c91b51b88667e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -147,6 +147,22 @@ public static long countRecordsSince(JavaSparkContext jsc, String basePath, SQLC } } + public static List getLatestBaseFiles(String basePath, FileSystem fs, + String... paths) { + List latestFiles = new ArrayList<>(); + try { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); + for (String path : paths) { + BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); + latestFiles.addAll(fileSystemView.getLatestBaseFiles().collect(Collectors.toList())); + } + } catch (Exception e) { + throw new HoodieException("Error reading hoodie table as a dataframe", e); + } + return latestFiles; + } + /** * Reads the paths under the a hoodie table out as a DataFrame. */ @@ -154,14 +170,9 @@ public static Dataset read(JavaSparkContext jsc, String basePath, SQLContex String... paths) { List filteredPaths = new ArrayList<>(); try { - HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true); - for (String path : paths) { - BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient, - metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path))); - List latestFiles = fileSystemView.getLatestBaseFiles().collect(Collectors.toList()); - for (HoodieBaseFile file : latestFiles) { - filteredPaths.add(file.getPath()); - } + List latestFiles = getLatestBaseFiles(basePath, fs, paths); + for (HoodieBaseFile file : latestFiles) { + filteredPaths.add(file.getPath()); } return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()])); } catch (Exception e) { diff --git a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc index fb41f6ef55855..b2444be84aa00 100644 --- a/hudi-common/src/main/avro/HoodieClusteringGroup.avsc +++ b/hudi-common/src/main/avro/HoodieClusteringGroup.avsc @@ -40,6 +40,11 @@ }], "default": null }, + { + "name":"numOutputFileGroups", + "type":["int", "null"], + "default": 1 + }, { "name":"version", "type":["int", "null"], diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index b1dcff90bca42..0557829d6ea3e 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 @@ -58,6 +58,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -308,17 +309,81 @@ public static GenericRecord rewriteRecordWithOnlyNewSchemaFields(GenericRecord r return rewrite(record, new LinkedHashSet<>(newSchema.getFields()), newSchema); } + private static void setDefaultVal(GenericRecord newRecord, Schema.Field f) { + if (f.defaultVal() instanceof JsonProperties.Null) { + newRecord.put(f.name(), null); + } else { + newRecord.put(f.name(), f.defaultVal()); + } + } + + /* + *

+   *  OldRecord:                     NewRecord:
+   *      field1 : String                field1 : String
+   *      field2 : record                field2 : record
+   *         field_21 : string              field_21 : string
+   *         field_22 : Integer             field_22 : Integer
+   *      field3: Integer                   field_23 : String
+   *                                       field_24 : Integer
+   *                                     field3: Integer
+   * 
+ *

+ * When a nested record has changed/evolved, newRecord.put(field2, oldRecord.get(field2)), is not sufficient. + * Requires a deep-copy/rewrite of the evolved field. + */ + private static Object rewriteEvolvedFields(Object datum, Schema newSchema) { + switch (newSchema.getType()) { + case RECORD: + if (!(datum instanceof GenericRecord)) { + return datum; + } + GenericRecord record = (GenericRecord) datum; + // if schema of the record being rewritten does not match + // with the new schema, some nested records with schema change + // will require rewrite. + if (!record.getSchema().equals(newSchema)) { + GenericRecord newRecord = new GenericData.Record(newSchema); + for (Schema.Field f : newSchema.getFields()) { + if (record.get(f.name()) == null) { + setDefaultVal(newRecord, f); + } else { + newRecord.put(f.name(), rewriteEvolvedFields(record.get(f.name()), f.schema())); + } + } + return newRecord; + } + return datum; + case UNION: + Integer idx = (newSchema.getTypes().get(0).getType() == Schema.Type.NULL) ? 1 : 0; + return rewriteEvolvedFields(datum, newSchema.getTypes().get(idx)); + case ARRAY: + List arrayValue = (List)datum; + List arrayCopy = new GenericData.Array( + arrayValue.size(), newSchema); + for (Object obj : arrayValue) { + arrayCopy.add(rewriteEvolvedFields(obj, newSchema.getElementType())); + } + return arrayCopy; + case MAP: + Map map = (Map)datum; + Map mapCopy = new HashMap<>(map.size()); + for (Map.Entry entry : map.entrySet()) { + mapCopy.put(entry.getKey(), rewriteEvolvedFields(entry.getValue(), newSchema.getValueType())); + } + return mapCopy; + default: + return datum; + } + } + private static GenericRecord rewrite(GenericRecord record, LinkedHashSet fieldsToWrite, Schema newSchema) { GenericRecord newRecord = new GenericData.Record(newSchema); for (Schema.Field f : fieldsToWrite) { if (record.get(f.name()) == null) { - if (f.defaultVal() instanceof JsonProperties.Null) { - newRecord.put(f.name(), null); - } else { - newRecord.put(f.name(), f.defaultVal()); - } + setDefaultVal(newRecord, f); } else { - newRecord.put(f.name(), record.get(f.name())); + newRecord.put(f.name(), rewriteEvolvedFields(record.get(f.name()), f.schema())); } } if (!GenericData.get().validate(newSchema, newRecord)) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java b/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java new file mode 100644 index 0000000000000..8f6da70844e07 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/SerializableSchema.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.config; + +import org.apache.avro.Schema; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * A wrapped Schema which can be serialized. + */ +public class SerializableSchema implements Serializable { + + private transient Schema schema; + + public SerializableSchema() { + } + + public SerializableSchema(Schema schema) { + this.schema = newCopy(schema); + } + + public SerializableSchema(SerializableSchema serializableSchema) { + this(serializableSchema.schema); + } + + public static Schema newCopy(Schema schemaObject) { + return new Schema.Parser().parse(schemaObject.toString()); + } + + public Schema get() { + return schema; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + writeObjectTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + readObjectFrom(in); + } + + // create a public write method for unit test + public void writeObjectTo(ObjectOutputStream out) throws IOException { + out.writeUTF(schema.toString()); + } + + // create a public read method for unit test + public void readObjectFrom(ObjectInputStream in) throws IOException { + schema = new Schema.Parser().parse(in.readUTF()); + } + + @Override + public String toString() { + return schema.toString(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java new file mode 100644 index 0000000000000..3d732fc7fa01a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ClusteringOperation.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.avro.model.HoodieSliceInfo; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * Encapsulates all the needed information about a clustering file slice. This is needed because spark serialization + * does not work with avro objects. + */ +public class ClusteringOperation implements Serializable { + + private String dataFilePath; + private List deltaFilePaths; + private String fileId; + private String partitionPath; + private String bootstrapFilePath; + private int version; + + public static ClusteringOperation create(HoodieSliceInfo sliceInfo) { + return new ClusteringOperation(sliceInfo.getDataFilePath(), new ArrayList<>(sliceInfo.getDeltaFilePaths()), sliceInfo.getFileId(), + sliceInfo.getPartitionPath(), sliceInfo.getBootstrapFilePath(), sliceInfo.getVersion()); + } + + // Only for serialization/de-serialization + @Deprecated + public ClusteringOperation() {} + + private ClusteringOperation(final String dataFilePath, final List deltaFilePaths, final String fileId, + final String partitionPath, final String bootstrapFilePath, final int version) { + this.dataFilePath = dataFilePath; + this.deltaFilePaths = deltaFilePaths; + this.fileId = fileId; + this.partitionPath = partitionPath; + this.bootstrapFilePath = bootstrapFilePath; + this.version = version; + } + + public String getDataFilePath() { + return this.dataFilePath; + } + + public void setDataFilePath(final String dataFilePath) { + this.dataFilePath = dataFilePath; + } + + public List getDeltaFilePaths() { + return this.deltaFilePaths; + } + + public void setDeltaFilePaths(final List deltaFilePaths) { + this.deltaFilePaths = deltaFilePaths; + } + + public String getFileId() { + return this.fileId; + } + + public void setFileId(final String fileId) { + this.fileId = fileId; + } + + public String getPartitionPath() { + return this.partitionPath; + } + + public void setPartitionPath(final String partitionPath) { + this.partitionPath = partitionPath; + } + + public String getBootstrapFilePath() { + return this.bootstrapFilePath; + } + + public void setBootstrapFilePath(final String bootstrapFilePath) { + this.bootstrapFilePath = bootstrapFilePath; + } + + public int getVersion() { + return this.version; + } + + public void setVersion(final int version) { + this.version = version; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ClusteringOperation that = (ClusteringOperation) o; + return getVersion() == that.getVersion() + && Objects.equals(getDataFilePath(), that.getDataFilePath()) + && Objects.equals(getDeltaFilePaths(), that.getDeltaFilePaths()) + && Objects.equals(getFileId(), that.getFileId()) + && Objects.equals(getPartitionPath(), that.getPartitionPath()) + && Objects.equals(getBootstrapFilePath(), that.getBootstrapFilePath()); + } + + @Override + public int hashCode() { + return Objects.hash(getDataFilePath(), getDeltaFilePaths(), getFileId(), getPartitionPath(), getBootstrapFilePath(), getVersion()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index 5f328a9bc69e4..f237156360847 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -42,6 +42,8 @@ public enum WriteOperationType { INSERT_OVERWRITE("insert_overwrite"), // cluster CLUSTER("cluster"), + // delete partition + DELETE_PARTITION("delete_partition"), // insert overwrite with dynamic partitioning INSERT_OVERWRITE_TABLE("insert_overwrite_table"), // used for old version @@ -74,8 +76,12 @@ public static WriteOperationType fromValue(String value) { return DELETE; case "insert_overwrite": return INSERT_OVERWRITE; + case "delete_partition": + return DELETE_PARTITION; case "insert_overwrite_table": return INSERT_OVERWRITE_TABLE; + case "cluster": + return CLUSTER; default: throw new HoodieException("Invalid value of Type."); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java new file mode 100644 index 0000000000000..3b73f41cbe539 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.log; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.SpillableMapUtils; +import org.apache.hudi.io.storage.HoodieFileReader; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice. + */ +public class HoodieFileSliceReader implements Iterator> { + private Iterator> recordsIterator; + + public static HoodieFileSliceReader getFileSliceReader( + HoodieFileReader baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass) throws IOException { + Iterator baseIterator = baseFileReader.getRecordIterator(schema); + while (baseIterator.hasNext()) { + GenericRecord record = (GenericRecord) baseIterator.next(); + HoodieRecord hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass); + scanner.processNextRecord(hoodieRecord); + } + return new HoodieFileSliceReader(scanner.iterator()); + } + + private HoodieFileSliceReader(Iterator> recordsItr) { + this.recordsIterator = recordsItr; + } + + @Override + public boolean hasNext() { + return recordsIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + return recordsIterator.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 99fd793e334c6..918d568b9faaf 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 @@ -306,7 +306,23 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI } /** - * Transition Clean State from inflight to Committed. + * Transition replace requested file to replace inflight. + * + * @param requestedInstant Requested instant + * @param data Extra Metadata + * @return inflight instant + */ + public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.getTimestamp()); + // Then write to timeline + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + /** + * Transition replace inflight to Committed. * * @param inflightInstant Inflight instant * @param data Extra Metadata @@ -321,6 +337,26 @@ public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightI return commitInstant; } + /** + * Revert replace requested State from inflight to requested. + * + * @param inflightInstant Inflight Instant + * @return requested instant + */ + public HoodieInstant revertReplaceCommitInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { transitionState(fromInstant, toInstant, data, false); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index ff251e3144487..06fe9619d8a5b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -299,6 +299,14 @@ static HoodieInstant getCompactionInflightInstant(final String timestamp) { return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp); } + static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp); + } + + static HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { + return new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, timestamp); + } + /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names * between inflight and completed instants (compaction <=> commit). diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index c0c88c04ac151..5cdb6fc193398 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -114,7 +114,11 @@ private static Stream> getFileGroupE new AbstractMap.SimpleEntry<>(entry.getLeft(), entry.getRight())); } - private static Stream getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) { + public static Stream getFileGroupsFromClusteringPlan(HoodieClusteringPlan clusteringPlan) { + return clusteringPlan.getInputGroups().stream().flatMap(ClusteringUtils::getFileGroupsFromClusteringGroup); + } + + public static Stream getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) { return group.getSlices().stream().map(slice -> new HoodieFileGroupId(slice.getPartitionPath(), slice.getFileId())); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index 49f1075508c86..326bf05277f0b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -25,6 +25,8 @@ */ public class StringUtils { + public static final String EMPTY_STRING = ""; + /** *

* Joins the elements of the provided array into a single String containing the provided list of elements. diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java index 40db67b50870e..50458bfe3b08d 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestHoodieAvroUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.avro; import org.apache.avro.JsonProperties; +import org.apache.avro.SchemaBuilder; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.exception.SchemaCompatibilityException; @@ -207,4 +208,82 @@ public void testAddingAndRemovingMetadataFields() { Schema schemaWithoutMetaCols = HoodieAvroUtils.removeMetadataFields(schemaWithMetaCols); assertEquals(schemaWithoutMetaCols.getFields().size(), NUM_FIELDS_IN_EXAMPLE_SCHEMA); } + + @Test + public void testRewriteToEvolvedNestedRecord() throws Exception { + // schema definition for inner record + Schema nestedSchema = SchemaBuilder.record("inner_rec").fields().requiredDouble("color_id").endRecord(); + Schema evolvedNestedSchema = SchemaBuilder.record("inner_rec").fields().requiredDouble("color_id") + .optionalString("color_name").endRecord(); + + // schema definition for outer record + Schema recordSchema = SchemaBuilder.record("outer_rec").fields().requiredDouble("timestamp") + .requiredString("_row_key").requiredString("non_pii_col").name("color_rec").type(nestedSchema) + .noDefault().requiredString("pii_col").endRecord(); + Schema evolvedRecordSchema = SchemaBuilder.record("outer_rec").fields().requiredDouble("timestamp") + .requiredString("_row_key").requiredString("non_pii_col").name("color_rec").type(evolvedNestedSchema) + .noDefault().requiredString("pii_col").endRecord(); + + // populate inner record, with fewer fields + GenericRecord nestedRec = new GenericData.Record(nestedSchema); + nestedRec.put("color_id", 55.5); + + // populate outer record + GenericRecord rec = new GenericData.Record(recordSchema); + rec.put("timestamp", 3.5); + rec.put("_row_key", "key1"); + rec.put("non_pii_col", "val1"); + rec.put("color_rec", nestedRec); + rec.put("pii_col", "val2"); + + // rewrite record with less number of fields into an evolved record (with optional fields added). + try { + GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(rec, evolvedRecordSchema); + assertEquals("val2", newRecord.get("pii_col")); + assertEquals(null, ((GenericRecord)newRecord.get("color_rec")).get("color_name")); + } catch (Exception e) { + e.printStackTrace(); + assertTrue(false, "Failed to rewrite Record"); + } + + } + + @Test + public void testRewriteToShorterRecord() throws Exception { + // schema definition for inner record + Schema nestedSchema = SchemaBuilder.record("inner_rec").fields().requiredDouble("color_id").endRecord(); + Schema largerNestedSchema = SchemaBuilder.record("inner_rec").fields().requiredDouble("color_id") + .requiredString("color_name").endRecord(); + + // schema definition for outer record + Schema recordSchema = SchemaBuilder.record("outer_rec").fields().requiredDouble("timestamp") + .requiredString("_row_key").requiredString("non_pii_col").name("color_rec").type(nestedSchema) + .noDefault().requiredString("pii_col").endRecord(); + Schema largerRecordSchema = SchemaBuilder.record("outer_rec").fields().requiredDouble("timestamp") + .requiredString("_row_key").requiredString("non_pii_col").name("color_rec").type(largerNestedSchema) + .noDefault().requiredString("pii_col").endRecord(); + + // populate larger inner record + GenericRecord nestedRec = new GenericData.Record(largerNestedSchema); + nestedRec.put("color_id", 55.5); + nestedRec.put("color_name", "blue"); + + // populate outer record, with larger inner record + GenericRecord largerRec = new GenericData.Record(largerRecordSchema); + largerRec.put("timestamp", 3.5); + largerRec.put("_row_key", "key1"); + largerRec.put("non_pii_col", "val1"); + largerRec.put("color_rec", nestedRec); + largerRec.put("pii_col", "val2"); + + // rewrite record with larger inner record to record with shorter inner record. + try { + GenericRecord shorterRec = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(largerRec, recordSchema); + assertEquals("val2", shorterRec.get("pii_col")); + assertEquals(null, ((GenericRecord)shorterRec.get("color_rec")).get("color_name")); + } catch (Exception e) { + e.printStackTrace(); + assertTrue(false, "Failed to rewrite Record"); + } + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.java new file mode 100644 index 0000000000000..72843a453c24c --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSerializableSchema.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.common.util; + +import org.apache.avro.Schema; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.config.SerializableSchema; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests serializable schema. + */ +public class TestSerializableSchema { + + @Test + public void testSerDeser() throws IOException { + verifySchema(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA)); + verifySchema(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA)); + verifySchema(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA); + verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA)); + verifySchema(HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS); + } + + private void verifySchema(Schema schema) throws IOException { + SerializableSchema serializableSchema = new SerializableSchema(schema); + assertEquals(schema, serializableSchema.get()); + assertTrue(schema != serializableSchema.get()); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(baos); + serializableSchema.writeObjectTo(oos); + oos.flush(); + oos.close(); + + byte[] bytesWritten = baos.toByteArray(); + SerializableSchema newSchema = new SerializableSchema(); + newSchema.readObjectFrom(new ObjectInputStream(new ByteArrayInputStream(bytesWritten))); + assertEquals(schema, newSchema.get()); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index b36885106cce0..9b0fbf9fc8b35 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -236,7 +236,7 @@ public static Option getAffectedPartitions(List commitsTo return false; }) .collect(Collectors.joining(",")); - return Option.of(incrementalInputPaths); + return StringUtils.isNullOrEmpty(incrementalInputPaths) ? Option.empty() : Option.of(incrementalInputPaths); } /** diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java index 27838897d5ec3..59214ac445049 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java @@ -117,7 +117,6 @@ public void testPendingCompactionWithActiveCommits() throws IOException { assertFalse(filteredTimeline.containsInstant(t5)); assertFalse(filteredTimeline.containsInstant(t6)); - // remove compaction instant and setup timeline again instants.remove(t3); timeline = new HoodieActiveTimeline(metaClient); @@ -239,6 +238,33 @@ public void testIncrementalSimple() throws IOException { "We should exclude commit 100 when returning incremental pull with start commit time as 100"); } + @Test + public void testIncrementalEmptyPartitions() throws IOException { + // initial commit + File partitionDir = InputFormatTestUtil.prepareTable(basePath, baseFileFormat, 10, "100"); + createCommitFile(basePath, "100", "2016/05/01"); + + // Add the paths + FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); + + InputFormatTestUtil.setupIncremental(jobConf, "000", 1); + + FileStatus[] files = inputFormat.listStatus(jobConf); + assertEquals(10, files.length, + "We should include only 1 commit 100 when returning incremental pull with start commit time as 100"); + ensureFilesInCommit("Pulling 1 commits from 000, should get us the 10 files from 100 commit", files, "100", 10); + + // Add new commit only to a new partition + partitionDir = InputFormatTestUtil.prepareTable(basePath, baseFileFormat, 10, "200"); + createCommitFile(basePath, "200", "2017/05/01"); + + InputFormatTestUtil.setupIncremental(jobConf, "100", 1); + files = inputFormat.listStatus(jobConf); + + assertEquals(0, files.length, + "We should exclude commit 200 when returning incremental pull with start commit time as 100 as filePaths does not include new partition"); + } + private void createCommitFile(java.nio.file.Path basePath, String commitNumber, String partitionPath) throws IOException { List writeStats = HoodieTestUtils.generateFakeHoodieWriteStat(1); @@ -355,7 +381,7 @@ public void testGetIncrementalTableNames() throws IOException { String incrementalMode1 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, expectedincrTables[0]); conf.set(incrementalMode1, HoodieHiveUtils.INCREMENTAL_SCAN_MODE); String incrementalMode2 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, expectedincrTables[1]); - conf.set(incrementalMode2,HoodieHiveUtils.INCREMENTAL_SCAN_MODE); + conf.set(incrementalMode2, HoodieHiveUtils.INCREMENTAL_SCAN_MODE); String incrementalMode3 = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, "db3.model_trips"); conf.set(incrementalMode3, HoodieHiveUtils.INCREMENTAL_SCAN_MODE.toLowerCase()); String defaultmode = String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, "db3.first_trips"); diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index a6cdd08478e1a..ff64ed1242fee 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -142,7 +142,9 @@ Start the Hudi Docker demo: docker/setup_demo.sh ``` -NOTE: We need to make a couple of environment changes for Hive 2.x support. This will be fixed once Hudi moves to Spark 3.x +NOTE: We need to make a couple of environment changes for Hive 2.x support. This will be fixed once Hudi moves to Spark 3.x. +Execute below if you are using Hudi query node in your dag. If not, below section is not required. +Also, for longer running tests, go to next section. ``` docker exec -it adhoc-2 bash @@ -214,7 +216,7 @@ spark-submit \ --conf spark.sql.catalogImplementation=hive \ --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ /opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ ---source-ordering-field timestamp \ +--source-ordering-field test_suite_source_ordering_field \ --use-deltastreamer \ --target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ --input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ @@ -253,7 +255,7 @@ spark-submit \ --conf spark.sql.catalogImplementation=hive \ --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ /opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ ---source-ordering-field timestamp \ +--source-ordering-field test_suite_source_ordering_field \ --use-deltastreamer \ --target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ --input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ @@ -267,3 +269,182 @@ spark-submit \ --table-type MERGE_ON_READ \ --compact-scheduling-minshare 1 ``` + +For long running test suite, validation has to be done differently. Idea is to run same dag in a repeated manner. +Hence "ValidateDatasetNode" is introduced which will read entire input data and compare it with hudi contents both via +spark datasource and hive table via spark sql engine. + +If you have "ValidateDatasetNode" in your dag, do not replace hive jars as instructed above. Spark sql engine does not +go well w/ hive2* jars. So, after running docker setup, just copy test.properties and your dag of interest and you are +good to go ahead. + +For repeated runs, two additional configs need to be set. "dag_rounds" and "dag_intermittent_delay_mins". +This means that your dag will be repeated for N times w/ a delay of Y mins between each round. + +Also, ValidateDatasetNode can be configured in two ways. Either with "delete_input_data: true" set or not set. +When "delete_input_data" is set for ValidateDatasetNode, once validation is complete, entire input data will be deleted. +So, suggestion is to use this ValidateDatasetNode as the last node in the dag with "delete_input_data". +Example dag: +``` + Insert + Upsert + ValidateDatasetNode with delete_input_data = true +``` + +If above dag is run with "dag_rounds" = 10 and "dag_intermittent_delay_mins" = 10, then this dag will run for 10 times +with 10 mins delay between every run. At the end of every run, records written as part of this round will be validated. +At the end of each validation, all contents of input are deleted. +For eg: incase of above dag, +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate with delete_input_data = true + Validates contents from batch1 and batch2 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch1 and batch2. +Round2: + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate with delete_input_data = true + Validates contents from batch3 and batch4 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch3 and batch4. +Round3: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate with delete_input_data = true + Validates contents from batch5 and batch6 are in hudi and ensures Row equality + Since "delete_input_data" is set, deletes contents from batch5 and batch6. +. +. +``` +If you wish to do a cumulative validation, do not set delete_input_data in ValidateDatasetNode. But remember that this +may not scale beyond certain point since input data as well as hudi content's keeps occupying the disk and grows for +every cycle. + +Lets see an example where you don't set "delete_input_data" as part of Validation. +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate: validates contents from batch1 and batch2 are in hudi and ensures Row equality +Round2: + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate: validates contents from batch1 to batch4 are in hudi and ensures Row equality +Round3: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate: validates contents from batch1 and batch6 are in hudi and ensures Row equality +. +. +``` + +You could also have validations in the middle of your dag and not set the "delete_input_data". But set it only in the +last node in the dag. +``` +Round1: + insert => inputPath/batch1 + upsert -> inputPath/batch2 + Validate: validates contents from batch1 and batch2 are in hudi and ensures Row equality + insert => inputPath/batch3 + upsert -> inputPath/batch4 + Validate with delete_input_data = true + Validates contents from batch1 to batch4 are in hudi and ensures Row equality + since "delete_input_data" is set to true, this node deletes contents from batch1 and batch4. +Round2: + insert => inputPath/batch5 + upsert -> inputPath/batch6 + Validate: validates contents from batch5 and batch6 are in hudi and ensures Row equality + insert => inputPath/batch7 + upsert -> inputPath/batch8 + Validate: validates contents from batch5 to batch8 are in hudi and ensures Row equality + since "delete_input_data" is set to true, this node deletes contents from batch5 to batch8. +Round3: + insert => inputPath/batch9 + upsert -> inputPath/batch10 + Validate: validates contents from batch9 and batch10 are in hudi and ensures Row equality + insert => inputPath/batch11 + upsert -> inputPath/batch12 + Validate with delete_input_data = true + Validates contents from batch9 to batch12 are in hudi and ensures Row equality + Set "delete_input_data" to true. so this node deletes contents from batch9 to batch12. +. +. +``` +Above dag was just an example for illustration purposes. But you can make it complex as per your needs. +``` + Insert + Upsert + Delete + Validate w/o deleting + Insert + Rollback + Validate w/o deleting + Upsert + Validate w/ deletion +``` +With this dag, you can set the two additional configs "dag_rounds" and "dag_intermittent_delay_mins" and have a long +running test suite. + +``` +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + Insert + Upsert + Delete + Validate w/o deleting + Insert + Rollback + Validate w/o deleting + Upsert + Validate w/ deletion + +``` + +Sample COW command with repeated runs. +``` +spark-submit \ +--packages org.apache.spark:spark-avro_2.11:2.4.0 \ +--conf spark.task.cpus=1 \ +--conf spark.executor.cores=1 \ +--conf spark.task.maxFailures=100 \ +--conf spark.memory.fraction=0.4 \ +--conf spark.rdd.compress=true \ +--conf spark.kryoserializer.buffer.max=2000m \ +--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ +--conf spark.memory.storageFraction=0.1 \ +--conf spark.shuffle.service.enabled=true \ +--conf spark.sql.hive.convertMetastoreParquet=false \ +--conf spark.driver.maxResultSize=12g \ +--conf spark.executor.heartbeatInterval=120s \ +--conf spark.network.timeout=600s \ +--conf spark.yarn.max.executor.failures=10 \ +--conf spark.sql.catalogImplementation=hive \ +--conf spark.driver.extraClassPath=/var/demo/jars/* \ +--conf spark.executor.extraClassPath=/var/demo/jars/* \ +--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \ +/opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \ +--source-ordering-field test_suite_source_ordering_field \ +--use-deltastreamer \ +--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ +--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ +--target-table table1 \ +--props test.properties \ +--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ +--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ +--input-file-size 125829120 \ +--workload-yaml-path file:/var/hoodie/ws/docker/demo/config/test-suite/complex-dag-cow.yaml \ +--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \ +--table-type COPY_ON_WRITE \ +--compact-scheduling-minshare 1 +``` + +A ready to use dag is available under docker/demo/config/test-suite/ that could give you an idea for long running +dags. +cow-per-round-mixed-validate.yaml + +As of now, "ValidateDatasetNode" uses spark data source and hive tables for comparison. Hence COW and real time view in +MOR can be tested. + + \ No newline at end of file diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java index 0387731d6b469..8d2f79defa8f2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java @@ -25,13 +25,13 @@ import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.SchemaProvider; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; /** - * Extends the {@link HoodieDeltaStreamer} to expose certain operations helpful in running the Test Suite. - * This is done to achieve 2 things 1) Leverage some components of {@link HoodieDeltaStreamer} 2) - * Piggyback on the suite to test {@link HoodieDeltaStreamer} + * Extends the {@link HoodieDeltaStreamer} to expose certain operations helpful in running the Test Suite. This is done to achieve 2 things 1) Leverage some components of {@link HoodieDeltaStreamer} + * 2) Piggyback on the suite to test {@link HoodieDeltaStreamer} */ public class HoodieDeltaStreamerWrapper extends HoodieDeltaStreamer { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index 7b3324e4b569e..b5037e9959cb3 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -18,13 +18,6 @@ package org.apache.hudi.integ.testsuite; -import com.beust.jcommander.JCommander; -import com.beust.jcommander.Parameter; -import java.io.IOException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hudi.DataSourceUtils; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; @@ -35,23 +28,30 @@ import org.apache.hudi.integ.testsuite.dag.DagUtils; import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator; +import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; -import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.keygen.BuiltinKeyGenerator; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; -import org.apache.hudi.utilities.schema.SchemaProvider; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; + /** - * This is the entry point for running a Hudi Test Suite. Although this class has similarities with - * {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency on the changes in - * DeltaStreamer. + * This is the entry point for running a Hudi Test Suite. Although this class has similarities with {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency + * on the changes in DeltaStreamer. */ public class HoodieTestSuiteJob { @@ -133,10 +133,10 @@ public static void main(String[] args) throws Exception { public WorkflowDag createWorkflowDag() throws IOException { WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils - .loadClass((this.cfg).workloadDagGenerator)).build() - : DagUtils.convertYamlPathToDag( - FSUtils.getFs(this.cfg.workloadYamlPath, jsc.hadoopConfiguration(), true), - this.cfg.workloadYamlPath); + .loadClass((this.cfg).workloadDagGenerator)).build() + : DagUtils.convertYamlPathToDag( + FSUtils.getFs(this.cfg.workloadYamlPath, jsc.hadoopConfiguration(), true), + this.cfg.workloadYamlPath); return workflowDag; } @@ -147,7 +147,7 @@ public void runTestSuite() { long startTime = System.currentTimeMillis(); WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession); writerContext.initContext(jsc); - DagScheduler dagScheduler = new DagScheduler(workflowDag, writerContext); + DagScheduler dagScheduler = new DagScheduler(workflowDag, writerContext, jsc); dagScheduler.schedule(); log.info("Finished scheduling all tasks, Time taken {}", System.currentTimeMillis() - startTime); } catch (Exception e) { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java index 81f406be3cc32..329ef16bd50d2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java @@ -87,6 +87,7 @@ public static class Config { private static String HIVE_LOCAL = "hive_local"; private static String REINIT_CONTEXT = "reinitialize_context"; private static String START_PARTITION = "start_partition"; + private static String DELETE_INPUT_DATA = "delete_input_data"; private Map configsMap; @@ -154,6 +155,10 @@ public boolean getReinitContext() { return Boolean.valueOf(configsMap.getOrDefault(REINIT_CONTEXT, false).toString()); } + public boolean isDeleteInputData() { + return Boolean.valueOf(configsMap.getOrDefault(DELETE_INPUT_DATA, false).toString()); + } + public Map getOtherConfigs() { if (configsMap == null) { return new HashMap<>(); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java index 24520a3626cf3..1e8acf580d962 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java @@ -18,12 +18,14 @@ package org.apache.hudi.integ.testsuite.converter; -import java.util.List; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.integ.testsuite.generator.LazyRecordGeneratorIterator; import org.apache.hudi.integ.testsuite.generator.UpdateGeneratorIterator; + +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; +import java.util.List; + /** * This converter creates an update {@link GenericRecord} from an existing {@link GenericRecord}. */ @@ -36,7 +38,7 @@ public class UpdateConverter implements Converter private final List recordKeyFields; private final int minPayloadSize; - public UpdateConverter(String schemaStr, int minPayloadSize, List partitionPathFields, + public UpdateConverter(String schemaStr, int minPayloadSize, List partitionPathFields, List recordKeyFields) { this.schemaStr = schemaStr; this.partitionPathFields = partitionPathFields; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java index d5358238d51a1..1211c0098d23a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java @@ -48,6 +48,15 @@ */ public class DagUtils { + public static final String DAG_NAME = "dag_name"; + public static final String DAG_ROUNDS = "dag_rounds"; + public static final String DAG_INTERMITTENT_DELAY_MINS = "dag_intermittent_delay_mins"; + public static final String DAG_CONTENT = "dag_content"; + + public static int DEFAULT_DAG_ROUNDS = 1; + public static int DEFAULT_INTERMITTENT_DELAY_MINS = 10; + public static String DEFAULT_DAG_NAME = "TestDagName"; + static final ObjectMapper MAPPER = new ObjectMapper(); /** @@ -62,15 +71,38 @@ public static WorkflowDag convertYamlPathToDag(FileSystem fs, String path) throw * Converts a YAML representation to {@link WorkflowDag}. */ public static WorkflowDag convertYamlToDag(String yaml) throws IOException { + int dagRounds = DEFAULT_DAG_ROUNDS; + int intermittentDelayMins = DEFAULT_INTERMITTENT_DELAY_MINS; + String dagName = DEFAULT_DAG_NAME; Map allNodes = new HashMap<>(); final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); final JsonNode jsonNode = yamlReader.readTree(yaml); Iterator> itr = jsonNode.fields(); while (itr.hasNext()) { Entry dagNode = itr.next(); - allNodes.put(dagNode.getKey(), convertJsonToDagNode(allNodes, dagNode.getKey(), dagNode.getValue())); + String key = dagNode.getKey(); + switch (key) { + case DAG_NAME: + dagName = dagNode.getValue().asText(); + break; + case DAG_ROUNDS: + dagRounds = dagNode.getValue().asInt(); + break; + case DAG_INTERMITTENT_DELAY_MINS: + intermittentDelayMins = dagNode.getValue().asInt(); + break; + case DAG_CONTENT: + JsonNode dagContent = dagNode.getValue(); + Iterator> contentItr = dagContent.fields(); + while(contentItr.hasNext()) { + Entry dagContentNode = contentItr.next(); + allNodes.put(dagContentNode.getKey(), convertJsonToDagNode(allNodes, dagContentNode.getKey(), dagContentNode.getValue())); + } + default: + break; + } } - return new WorkflowDag(findRootNodes(allNodes)); + return new WorkflowDag(dagName, dagRounds, intermittentDelayMins, findRootNodes(allNodes)); } /** diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java index ad6e9cb0cb1b9..1fe2294423121 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java @@ -18,8 +18,6 @@ package org.apache.hudi.integ.testsuite.dag; -import java.util.ArrayList; -import java.util.List; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; @@ -27,9 +25,11 @@ import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode; import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode; +import java.util.ArrayList; +import java.util.List; + /** - * An example of how to generate a workflow dag programmatically. This is also used as the default workflow dag if - * none is provided. + * An example of how to generate a workflow dag programmatically. This is also used as the default workflow dag if none is provided. */ public class SimpleWorkflowDagGenerator implements WorkflowDagGenerator { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java index e9171fc4774d0..f622bb7a7e448 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java @@ -18,20 +18,47 @@ package org.apache.hudi.integ.testsuite.dag; -import java.util.List; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; +import java.util.List; + +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_DAG_NAME; +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_DAG_ROUNDS; +import static org.apache.hudi.integ.testsuite.dag.DagUtils.DEFAULT_INTERMITTENT_DELAY_MINS; + /** * Workflow dag that encapsulates all execute nodes. */ public class WorkflowDag { + private String dagName; + private int rounds; + private int intermittentDelayMins; private List> nodeList; public WorkflowDag(List> nodeList) { + this(DEFAULT_DAG_NAME, DEFAULT_DAG_ROUNDS, DEFAULT_INTERMITTENT_DELAY_MINS, nodeList); + } + + public WorkflowDag(String dagName, int rounds, int intermittentDelayMins, List> nodeList) { + this.dagName = dagName; + this.rounds = rounds; + this.intermittentDelayMins = intermittentDelayMins; this.nodeList = nodeList; } + public String getDagName() { + return dagName; + } + + public int getRounds() { + return rounds; + } + + public int getIntermittentDelayMins() { + return intermittentDelayMins; + } + public List> getNodeList() { return nodeList; } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java index e457f0a8daca7..650ab1eeaf4c2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WriterContext.java @@ -21,15 +21,16 @@ import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; +import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode; import org.apache.hudi.keygen.BuiltinKeyGenerator; -import org.apache.hudi.integ.testsuite.generator.DeltaGenerator; -import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -38,8 +39,7 @@ import java.util.Map; /** - * WriterContext wraps the delta writer/data generator related configuration needed - * to init/reinit. + * WriterContext wraps the delta writer/data generator related configuration needed to init/reinit. */ public class WriterContext { @@ -53,8 +53,9 @@ public class WriterContext { private BuiltinKeyGenerator keyGenerator; private transient SparkSession sparkSession; private transient JavaSparkContext jsc; + public WriterContext(JavaSparkContext jsc, TypedProperties props, HoodieTestSuiteConfig cfg, - BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { + BuiltinKeyGenerator keyGenerator, SparkSession sparkSession) { this.cfg = cfg; this.props = props; this.keyGenerator = keyGenerator; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java index df54b4c811989..05ac242a5cefc 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java @@ -41,6 +41,17 @@ public abstract class DagNode implements Comparable> { protected Config config; private boolean isCompleted; + public DagNode clone() { + List> tempChildNodes = new ArrayList<>(); + for(DagNode dagNode: childNodes) { + tempChildNodes.add(dagNode.clone()); + } + this.childNodes = tempChildNodes; + this.result = null; + this.isCompleted = false; + return this; + } + public DagNode addChildNode(DagNode childNode) { childNode.getParentNodes().add(this); getChildNodes().add(childNode); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java new file mode 100644 index 0000000000000..c0671e8abb1ef --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DelayNode.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes; + +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Delay Node to add delays between each group of test runs. + */ +public class DelayNode extends DagNode { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + private int delayMins; + + public DelayNode(int delayMins) { + this.delayMins = delayMins; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + log.warn("Waiting for "+ delayMins+" mins before going for next test run"); + Thread.sleep(delayMins * 60 * 1000); + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java new file mode 100644 index 0000000000000..12fc52529a0ab --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; + +/** + * This nodes validates contents from input path are in tact with Hudi. This nodes uses spark datasource for comparison purposes. By default no configs are required for this node. But there is an + * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. + * README has more details under docker set up for usages of this node. + */ +public class ValidateDatasetNode extends DagNode { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + + public ValidateDatasetNode(Config config) { + this.config = config; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + + SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); + + // todo: Fix partitioning schemes. For now, assumes data based partitioning. + String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; + String hudiPath = context.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/*/*/*"; + log.warn("ValidateDataset Node: Input path " + inputPath + ", hudi path " + hudiPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } + + String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); + String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()); + // todo: fix hard coded fields from configs. + // read input and resolve insert, updates, etc. + Dataset inputDf = session.read().format("avro").load(inputPath); + ExpressionEncoder encoder = getEncoder(inputDf.schema()); + Dataset inputSnapshotDf = inputDf.groupByKey( + (MapFunction) value -> partitionPathField + "+" + recordKeyField, Encoders.STRING()) + .reduceGroups((ReduceFunction) (v1, v2) -> { + int ts1 = v1.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + int ts2 = v2.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + if (ts1 > ts2) { + return v1; + } else { + return v2; + } + }) + .map((MapFunction, Row>) value -> value._2, encoder) + .filter("_hoodie_is_deleted is NULL"); + + // read from hudi and remove meta columns. + Dataset hudiDf = session.read().format("hudi").load(hudiPath); + Dataset trimmedDf = hudiDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD); + + Dataset intersectionDf = inputSnapshotDf.intersect(trimmedDf); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed. Total count in hudi " + trimmedDf.count() + ", input df count " + inputSnapshotDf.count()); + throw new AssertionError("Hudi contents does not match contents input data. "); + } + + String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY()); + String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()); + log.warn("Validating hive table with db : " + database + " and table : " + tableName); + Dataset cowDf = session.sql("SELECT * FROM " + database + "." + tableName); + Dataset trimmedCowDf = cowDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD); + intersectionDf = inputSnapshotDf.intersect(trimmedDf); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed for COW hive table. Total count in hudi " + trimmedCowDf.count() + ", input df count " + inputSnapshotDf.count()); + throw new AssertionError("Hudi hive table contents does not match contents input data. "); + } + + // if delete input data is enabled, erase input data. + if (config.isDeleteInputData()) { + // clean up input data for current group of writes. + inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Micro batch to be deleted " + fileStatus.getPath().toString()); + fs.delete(fileStatus.getPath(), true); + } + } + } + + private ExpressionEncoder getEncoder(StructType schema) { + List attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream() + .map(Attribute::toAttribute).collect(Collectors.toList()); + return RowEncoder.apply(schema) + .resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(), + SimpleAnalyzer$.MODULE$); + } +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java index 5c70ea164e873..d4074bcccdae0 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java @@ -23,8 +23,10 @@ import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.WriterContext; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; +import org.apache.hudi.integ.testsuite.dag.nodes.DelayNode; import org.apache.hudi.metrics.Metrics; +import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,9 +52,9 @@ public class DagScheduler { private WorkflowDag workflowDag; private ExecutionContext executionContext; - public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext) { + public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext, JavaSparkContext jsc) { this.workflowDag = workflowDag; - this.executionContext = new ExecutionContext(null, writerContext); + this.executionContext = new ExecutionContext(jsc, writerContext); } /** @@ -63,7 +65,7 @@ public DagScheduler(WorkflowDag workflowDag, WriterContext writerContext) { public void schedule() throws Exception { ExecutorService service = Executors.newFixedThreadPool(2); try { - execute(service, workflowDag.getNodeList()); + execute(service, workflowDag); service.shutdown(); } finally { if (!service.isShutdown()) { @@ -77,33 +79,47 @@ public void schedule() throws Exception { * Method to start executing the nodes in workflow DAGs. * * @param service ExecutorService - * @param nodes Nodes to be executed + * @param workflowDag instance of workflow dag that needs to be executed * @throws Exception will be thrown if ant error occurred */ - private void execute(ExecutorService service, List nodes) throws Exception { + private void execute(ExecutorService service, WorkflowDag workflowDag) throws Exception { // Nodes at the same level are executed in parallel - Queue queue = new PriorityQueue<>(nodes); log.info("Running workloads"); + List nodes = workflowDag.getNodeList(); + int curRound = 1; do { - List futures = new ArrayList<>(); - Set childNodes = new HashSet<>(); - while (queue.size() > 0) { - DagNode nodeToExecute = queue.poll(); - log.info("Node to execute in dag scheduler " + nodeToExecute.getConfig().toString()); - futures.add(service.submit(() -> executeNode(nodeToExecute))); - if (nodeToExecute.getChildNodes().size() > 0) { - childNodes.addAll(nodeToExecute.getChildNodes()); - } + log.warn("==================================================================="); + log.warn("Running workloads for round num " + curRound); + log.warn("==================================================================="); + Queue queue = new PriorityQueue<>(); + for (DagNode dagNode : nodes) { + queue.add(dagNode.clone()); } - queue.addAll(childNodes); - childNodes.clear(); - for (Future future : futures) { - future.get(1, TimeUnit.HOURS); + do { + List futures = new ArrayList<>(); + Set childNodes = new HashSet<>(); + while (queue.size() > 0) { + DagNode nodeToExecute = queue.poll(); + log.warn("Executing node \"" + nodeToExecute.getConfig().getOtherConfigs().get(CONFIG_NAME) + "\" :: " + nodeToExecute.getConfig()); + futures.add(service.submit(() -> executeNode(nodeToExecute))); + if (nodeToExecute.getChildNodes().size() > 0) { + childNodes.addAll(nodeToExecute.getChildNodes()); + } + } + queue.addAll(childNodes); + childNodes.clear(); + for (Future future : futures) { + future.get(1, TimeUnit.HOURS); + } + } while (queue.size() > 0); + log.info("Finished workloads for round num " + curRound); + if (curRound < workflowDag.getRounds()) { + new DelayNode(workflowDag.getIntermittentDelayMins()).execute(executionContext); } // After each level, report and flush the metrics Metrics.flush(); - } while (queue.size() > 0); + } while (curRound++ < workflowDag.getRounds()); log.info("Finished workloads"); } @@ -119,7 +135,6 @@ private void executeNode(DagNode node) { try { int repeatCount = node.getConfig().getRepeatCount(); while (repeatCount > 0) { - log.warn("executing node: \"" + node.getConfig().getOtherConfigs().get(CONFIG_NAME) + "\" of type: " + node.getClass() + " :: " + node.getConfig().toString()); node.execute(executionContext); log.info("Finished executing {}", node.getName()); repeatCount--; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java index 53af8eb74068e..6242cbfc7b10b 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java @@ -41,6 +41,10 @@ import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.converter.Converter; import org.apache.hudi.integ.testsuite.converter.DeleteConverter; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; import org.apache.hudi.integ.testsuite.converter.UpdateConverter; import org.apache.hudi.integ.testsuite.reader.DFSAvroDeltaInputReader; import org.apache.hudi.integ.testsuite.reader.DFSHoodieDatasetInputReader; @@ -51,6 +55,7 @@ import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory; import org.apache.hudi.keygen.BuiltinKeyGenerator; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; @@ -58,6 +63,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.StreamSupport; import scala.Tuple2; @@ -77,7 +93,7 @@ public class DeltaGenerator implements Serializable { private int batchId; public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession, - String schemaStr, BuiltinKeyGenerator keyGenerator) { + String schemaStr, BuiltinKeyGenerator keyGenerator) { this.deltaOutputConfig = deltaOutputConfig; this.jsc = jsc; this.sparkSession = sparkSession; @@ -167,7 +183,6 @@ public JavaRDD generateUpdates(Config config) throws IOException log.info("Repartitioning records into " + numPartition + " partitions for updates"); adjustedRDD = adjustedRDD.repartition(numPartition); log.info("Repartitioning records done for updates"); - UpdateConverter converter = new UpdateConverter(schemaStr, config.getRecordSize(), partitionPathFieldNames, recordRowKeyFieldNames); JavaRDD updates = converter.convert(adjustedRDD); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java index 256dfa49ebf6c..5477371a1defa 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java @@ -20,6 +20,11 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.List; import java.util.HashSet; import java.util.Iterator; @@ -67,7 +72,7 @@ public GenericRecord next() { lastRecord = record; return record; } else { - return this.generator.randomize(lastRecord, partitionPathFieldNames); + return this.generator.randomize(lastRecord, this.partitionPathFieldNames); } } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java index 7d5ca081444f0..49a5f312ecabe 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 @@ -134,12 +134,16 @@ public GenericRecord getNewPayload(Set partitionPathFieldNames) { protected GenericRecord create(Schema schema, Set partitionPathFieldNames) { GenericRecord result = new GenericData.Record(schema); for (Schema.Field f : schema.getFields()) { - if (isPartialLongField(f, partitionPathFieldNames)) { - // This is a long field used as partition field. Set it to seconds since epoch. - long value = TimeUnit.SECONDS.convert(partitionIndex, TimeUnit.DAYS); - result.put(f.name(), (long) value); + if (f.name().equals(DEFAULT_HOODIE_IS_DELETED_COL)) { + result.put(f.name(), false); } else { - result.put(f.name(), typeConvert(f)); + if (isPartialLongField(f, partitionPathFieldNames)) { + // This is a long field used as partition field. Set it to seconds since epoch. + long value = TimeUnit.SECONDS.convert(partitionIndex, TimeUnit.DAYS); + result.put(f.name(), (long) value); + } else { + result.put(f.name(), typeConvert(f)); + } } } return result; diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java index 51b1fd9ed48dc..89cda658e12cc 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java @@ -18,16 +18,16 @@ package org.apache.hudi.integ.testsuite.generator; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + /** * A lazy update payload generator to generate {@link GenericRecord}s lazily. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java index b7d71f583777a..94ff3a3ea2327 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java @@ -18,20 +18,6 @@ package org.apache.hudi.integ.testsuite.helpers; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FsStatus; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -40,13 +26,26 @@ import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob; import org.apache.hudi.utilities.sources.helpers.DFSPathSelector; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + /** * A custom dfs path selector used only for the hudi test suite. To be used only if workload is not run inline. */ public class DFSTestSuitePathSelector extends DFSPathSelector { + private static volatile Logger log = LoggerFactory.getLogger(HoodieTestSuiteJob.class); public DFSTestSuitePathSelector(TypedProperties props, Configuration hadoopConf) { @@ -67,6 +66,7 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( lastBatchId = 0; nextBatchId = 1; } + // obtain all eligible files for the batch List eligibleFiles = new ArrayList<>(); FileStatus[] fileStatuses = fs.globStatus( @@ -87,7 +87,8 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( if (!fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream() .anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) { continue; - } else if (fileStatus.getPath().getName().compareTo(lastBatchId.toString()) > 0) { + } else if (Integer.parseInt(fileStatus.getPath().getName()) > lastBatchId && Integer.parseInt(fileStatus.getPath() + .getName()) <= nextBatchId) { RemoteIterator files = fs.listFiles(fileStatus.getPath(), true); while (files.hasNext()) { eligibleFiles.add(files.next()); @@ -95,7 +96,6 @@ public Pair, String> getNextFilePathsAndMaxModificationTime( } } - log.info("Reading " + eligibleFiles.size() + " files. "); // no data to readAvro if (eligibleFiles.size() == 0) { return new ImmutablePair<>(Option.empty(), diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index bc7803d9d1fc5..43d5fdefaf461 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -18,26 +18,6 @@ package org.apache.hudi.integ.testsuite.reader; -import static java.util.Map.Entry.comparingByValue; -import static java.util.stream.Collectors.toMap; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; @@ -51,6 +31,12 @@ import org.apache.hudi.common.util.ParquetReaderIterator; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieMemoryConfig; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; import org.apache.spark.api.java.JavaPairRDD; @@ -59,11 +45,27 @@ import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + import scala.Tuple2; +import static java.util.Map.Entry.comparingByValue; +import static java.util.stream.Collectors.toMap; + /** - * This class helps to generate updates from an already existing hoodie dataset. It supports generating updates in - * across partitions, files and records. + * This class helps to generate updates from an already existing hoodie dataset. It supports generating updates in across partitions, files and records. */ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader { @@ -148,16 +150,22 @@ private JavaRDD fetchRecordsFromDataset(Option numPartit long recordsInSingleFile = iteratorSize(readParquetOrLogFiles(getSingleSliceFromRDD(partitionToFileSlice))); int numFilesToUpdate; long numRecordsToUpdatePerFile; - if (!numFiles.isPresent() || numFiles.get() == 0) { + if (!numFiles.isPresent() || numFiles.get() <= 0) { // If num files are not passed, find the number of files to update based on total records to update and records // per file - numFilesToUpdate = (int)Math.ceil((double)numRecordsToUpdate.get() / recordsInSingleFile); - // recordsInSingleFile is not average so we still need to account for bias is records distribution - // in the files. Limit to the maximum number of files available. - int totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get(); - numFilesToUpdate = Math.min(numFilesToUpdate, totalExistingFilesCount); - log.info("Files to update {}", numFilesToUpdate); - numRecordsToUpdatePerFile = recordsInSingleFile; + numFilesToUpdate = (int) Math.floor((double) numRecordsToUpdate.get() / recordsInSingleFile); + if (numFilesToUpdate > 0) { + // recordsInSingleFile is not average so we still need to account for bias is records distribution + // in the files. Limit to the maximum number of files available. + int totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get(); + numFilesToUpdate = Math.min(numFilesToUpdate, totalExistingFilesCount); + log.info("Files to update {}, records to update per file {}", numFilesToUpdate, recordsInSingleFile); + numRecordsToUpdatePerFile = recordsInSingleFile; + } else { + numFilesToUpdate = 1; + numRecordsToUpdatePerFile = numRecordsToUpdate.get(); + log.info("Total records passed in < records in single file. Hence setting numFilesToUpdate to 1 and numRecordsToUpdate to {} ", numRecordsToUpdatePerFile); + } } else { // If num files is passed, find the number of records per file based on either percentage or total records to // update and num files passed @@ -171,6 +179,7 @@ private JavaRDD fetchRecordsFromDataset(Option numPartit partitionPaths.size(), numFilesToUpdate, partitionToFileIdCountMap); JavaRDD updates = projectSchema(generateUpdates(adjustedPartitionToFileIdCountMap, partitionToFileSlice, numFilesToUpdate, (int) numRecordsToUpdatePerFile)); + if (numRecordsToUpdate.isPresent() && numFiles.isPresent() && numFiles.get() != 0 && numRecordsToUpdate.get() != numRecordsToUpdatePerFile * numFiles.get()) { long remainingRecordsToAdd = (numRecordsToUpdate.get() - (numRecordsToUpdatePerFile * numFiles.get())); @@ -215,7 +224,7 @@ private Map getFilesToReadPerPartition(JavaPairRDD adjustedPartitionToFileIdCountMap = new HashMap<>(); partitionToFileIdCountSortedMap.entrySet().stream().forEach(e -> { if (e.getValue() <= numFilesPerPartition) { @@ -283,9 +292,7 @@ private Iterator readParquetOrLogFiles(FileSlice fileSlice) throw } /** - * Returns the number of elements remaining in {@code iterator}. The iterator - * will be left exhausted: its {@code hasNext()} method will return - * {@code false}. + * Returns the number of elements remaining in {@code iterator}. The iterator will be left exhausted: its {@code hasNext()} method will return {@code false}. */ private static int iteratorSize(Iterator iterator) { int count = 0; @@ -297,11 +304,8 @@ private static int iteratorSize(Iterator iterator) { } /** - * Creates an iterator returning the first {@code limitSize} elements of the - * given iterator. If the original iterator does not contain that many - * elements, the returned iterator will have the same behavior as the original - * iterator. The returned iterator supports {@code remove()} if the original - * iterator does. + * Creates an iterator returning the first {@code limitSize} elements of the given iterator. If the original iterator does not contain that many elements, the returned iterator will have the same + * behavior as the original iterator. The returned iterator supports {@code remove()} if the original iterator does. * * @param iterator the iterator to limit * @param limitSize the maximum number of elements in the returned iterator diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java new file mode 100644 index 0000000000000..2de945286d910 --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/SchemaUtils.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.schema; + +public class SchemaUtils { + + public static final String SOURCE_ORDERING_FIELD = "test_suite_source_ordering_field"; + +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java new file mode 100644 index 0000000000000..e67c5afae80dc --- /dev/null +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.integ.testsuite.dag.WriterContext; +import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Appends source ordering field to both source and target schemas. This is required to assist in validation to differentiate records written in different batches. + */ +public class TestSuiteFileBasedSchemaProvider extends FilebasedSchemaProvider { + + protected static Logger log = LogManager.getLogger(WriterContext.class); + + public TestSuiteFileBasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + this.sourceSchema = addSourceOrderingFieldToSchema(sourceSchema); + this.targetSchema = addSourceOrderingFieldToSchema(targetSchema); + } + + private Schema addSourceOrderingFieldToSchema(Schema schema) { + List fields = new ArrayList<>(); + for (Schema.Field field : schema.getFields()) { + Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal()); + for (Map.Entry prop : field.getObjectProps().entrySet()) { + newField.addProp(prop.getKey(), prop.getValue()); + } + fields.add(newField); + } + Schema.Field sourceOrderingField = + new Schema.Field(SchemaUtils.SOURCE_ORDERING_FIELD, Schema.create(Type.INT), "", 0); + fields.add(sourceOrderingField); + Schema mergedSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false); + mergedSchema.setFields(fields); + return mergedSchema; + } + +} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java index 65e4ee13cf55f..4bd096ae0cbde 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java @@ -18,6 +18,8 @@ package org.apache.hudi.integ.testsuite.writer; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + import org.apache.avro.generic.GenericRecord; import java.io.IOException; @@ -30,22 +32,29 @@ */ public class DFSDeltaWriterAdapter implements DeltaWriterAdapter { - private DeltaInputWriter deltaInputGenerator; + private DeltaInputWriter deltaInputWriter; private List metrics = new ArrayList<>(); + private int preCombineFieldVal = 0; + + public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter, int preCombineFieldVal) { + this.deltaInputWriter = deltaInputWriter; + this.preCombineFieldVal = preCombineFieldVal; + } - public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputGenerator) { - this.deltaInputGenerator = deltaInputGenerator; + public DFSDeltaWriterAdapter(DeltaInputWriter deltaInputWriter) { + this.deltaInputWriter = deltaInputWriter; } @Override public List write(Iterator input) throws IOException { while (input.hasNext()) { GenericRecord next = input.next(); - if (this.deltaInputGenerator.canWrite()) { - this.deltaInputGenerator.writeData(next); - } else if (input.hasNext()) { + next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal); + if (this.deltaInputWriter.canWrite()) { + this.deltaInputWriter.writeData(next); + } else { rollOver(); - this.deltaInputGenerator.writeData(next); + this.deltaInputWriter.writeData(next); } } close(); @@ -54,11 +63,11 @@ public List write(Iterator input) throws IOExcep public void rollOver() throws IOException { close(); - this.deltaInputGenerator = this.deltaInputGenerator.getNewWriter(); + this.deltaInputWriter = this.deltaInputWriter.getNewWriter(); } private void close() throws IOException { - this.deltaInputGenerator.close(); - this.metrics.add(this.deltaInputGenerator.getDeltaWriteStats()); + this.deltaInputWriter.close(); + this.metrics.add(this.deltaInputWriter.getDeltaWriteStats()); } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java index b4d9b9f8956d8..a00e8e15d401a 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java @@ -18,16 +18,17 @@ package org.apache.hudi.integ.testsuite.writer; -import java.io.IOException; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; import org.apache.hudi.integ.testsuite.reader.DeltaInputType; +import org.apache.avro.generic.GenericRecord; + +import java.io.IOException; + /** - * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputMode} and - * {@link DeltaInputType}. + * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputMode} and {@link DeltaInputType}. */ public class DeltaWriterFactory { @@ -44,9 +45,9 @@ public static DeltaWriterAdapter getDeltaWriterAdapter(DeltaConfig config, Integ DeltaInputWriter fileDeltaInputGenerator = new AvroFileDeltaInputWriter( dfsDeltaConfig.getConfiguration(), StringUtils - .join(new String[]{dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()}, + .join(new String[] {dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()}, "/"), dfsDeltaConfig.getSchemaStr(), dfsDeltaConfig.getMaxFileSize()); - return new DFSDeltaWriterAdapter(fileDeltaInputGenerator); + return new DFSDeltaWriterAdapter(fileDeltaInputGenerator, batchId); default: throw new IllegalArgumentException("Invalid delta input format " + config.getDeltaInputType()); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java index 1e5ca6886c8e6..82350999ea42c 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java @@ -18,18 +18,19 @@ package org.apache.hudi.integ.testsuite.configuration; -import static junit.framework.Assert.assertTrue; -import static junit.framework.TestCase.assertEquals; - -import java.util.ArrayList; -import java.util.List; - +import org.apache.hudi.integ.testsuite.dag.WorkflowDag; import org.apache.hudi.integ.testsuite.dag.nodes.DagNode; import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode; import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode; -import org.apache.hudi.integ.testsuite.dag.WorkflowDag; + import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.Assert.assertTrue; +import static junit.framework.TestCase.assertEquals; + /** * Unit test for the build process of {@link DagNode} and {@link WorkflowDag}. */ diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java index d94174471bd64..70e6da7d3c49a 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java @@ -47,6 +47,9 @@ public void testConvertDagToYaml() throws Exception { public void testConvertYamlToDag() throws Exception { WorkflowDag dag = DagUtils.convertYamlToDag(UtilitiesTestBase.Helpers .readFileFromAbsolutePath((System.getProperty("user.dir") + "/.." + COW_DAG_DOCKER_DEMO_RELATIVE_PATH))); + assertEquals(dag.getDagName(), "unit-test-cow-dag"); + assertEquals(dag.getRounds(), 1); + assertEquals(dag.getIntermittentDelayMins(), 10); assertEquals(dag.getNodeList().size(), 1); Assertions.assertEquals(((DagNode) dag.getNodeList().get(0)).getParentNodes().size(), 0); assertEquals(((DagNode) dag.getNodeList().get(0)).getChildNodes().size(), 1); diff --git a/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml b/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml index 96a6c825a98d0..23691659cab24 100644 --- a/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml +++ b/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml @@ -13,58 +13,62 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -first_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 2 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_insert -first_rollback: - config: - deps: second_insert - type: RollbackNode -third_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_rollback -first_upsert: - config: - record_size: 70000 - num_partitions_upsert: 1 - repeat_count: 1 - num_records_upsert: 100 - type: UpsertNode - deps: third_insert -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - hive_props: - prop2: "set spark.yarn.queue=" - prop3: "set hive.strict.checks.large.query=false" - prop4: "set hive.stats.autogather=false" - hive_queries: - query1: "select count(*) from testdb1.table1" - result1: 300 - query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" - result2: 0 - type: HiveQueryNode - deps: first_hive_sync \ No newline at end of file +dag_name: unit-test-cow-dag +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_insert + first_rollback: + config: + deps: second_insert + type: RollbackNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_rollback + first_upsert: + config: + record_size: 70000 + num_partitions_upsert: 1 + repeat_count: 1 + num_records_upsert: 100 + type: UpsertNode + deps: third_insert + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + hive_props: + prop2: "set spark.yarn.queue=" + prop3: "set hive.strict.checks.large.query=false" + prop4: "set hive.stats.autogather=false" + hive_queries: + query1: "select count(*) from testdb1.table1" + result1: 300 + query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" + result2: 0 + type: HiveQueryNode + deps: first_hive_sync \ No newline at end of file diff --git a/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml b/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml index 96a6c825a98d0..2ba42455d4874 100644 --- a/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml +++ b/hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml @@ -13,58 +13,62 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -first_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 2 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_insert -first_rollback: - config: - deps: second_insert - type: RollbackNode -third_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_rollback -first_upsert: - config: - record_size: 70000 - num_partitions_upsert: 1 - repeat_count: 1 - num_records_upsert: 100 - type: UpsertNode - deps: third_insert -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -first_hive_query: - config: - hive_props: - prop2: "set spark.yarn.queue=" - prop3: "set hive.strict.checks.large.query=false" - prop4: "set hive.stats.autogather=false" - hive_queries: - query1: "select count(*) from testdb1.table1" - result1: 300 - query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" - result2: 0 - type: HiveQueryNode - deps: first_hive_sync \ No newline at end of file +dag_name: unit-test-mor-dag +dag_rounds: 1 +dag_intermittent_delay_mins: 10 +dag_content: + first_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 2 + num_records_insert: 100 + type: InsertNode + deps: none + second_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_insert + first_rollback: + config: + deps: second_insert + type: RollbackNode + third_insert: + config: + record_size: 70000 + num_partitions_insert: 1 + repeat_count: 1 + num_records_insert: 100 + type: InsertNode + deps: first_rollback + first_upsert: + config: + record_size: 70000 + num_partitions_upsert: 1 + repeat_count: 1 + num_records_upsert: 100 + type: UpsertNode + deps: third_insert + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_upsert + first_hive_query: + config: + hive_props: + prop2: "set spark.yarn.queue=" + prop3: "set hive.strict.checks.large.query=false" + prop4: "set hive.stats.autogather=false" + hive_queries: + query1: "select count(*) from testdb1.table1" + result1: 300 + query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" + result2: 0 + type: HiveQueryNode + deps: first_hive_sync \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml index af6403dc59c8b..763d3b2b71aff 100644 --- a/hudi-spark-datasource/hudi-spark-common/pom.xml +++ b/hudi-spark-datasource/hudi-spark-common/pom.xml @@ -173,6 +173,38 @@ org.apache.spark spark-sql_${scala.binary.version} + + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-client + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + + org.junit.jupiter + junit-jupiter-api + test + diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java new file mode 100644 index 0000000000000..e75c9a213f36d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseDefaultSource.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; + +/** + * Base class for DefaultSource used by Spark datasource v2. + */ +public class BaseDefaultSource { + + protected SparkSession sparkSession = null; + protected Configuration configuration = null; + + protected SparkSession getSparkSession() { + if (sparkSession == null) { + sparkSession = SparkSession.builder().getOrCreate(); + } + return sparkSession; + } + + protected Configuration getConfiguration() { + if (configuration == null) { + this.configuration = getSparkSession().sparkContext().hadoopConfiguration(); + } + return configuration; + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java new file mode 100644 index 0000000000000..88a7921236a58 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BaseWriterCommitMessage.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; + +import java.util.Arrays; +import java.util.List; + +/** + * Base class for HoodieWriterCommitMessage used by Spark datasource v2. + */ +public class BaseWriterCommitMessage { + + private List writeStatuses; + + public BaseWriterCommitMessage(List writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public List getWriteStatuses() { + return writeStatuses; + } + + @Override + public String toString() { + return "HoodieWriterCommitMessage{" + "writeStatuses=" + Arrays.toString(writeStatuses.toArray()) + '}'; + } +} diff --git a/hudi-spark-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 new file mode 100644 index 0000000000000..eb26c4f3209c4 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.io.HoodieRowCreateHandle; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +/** + * Helper class for HoodieBulkInsertDataInternalWriter used by Spark datasource v2. + */ +public class BulkInsertDataInternalWriterHelper { + + private static final Logger LOG = LogManager.getLogger(BulkInsertDataInternalWriterHelper.class); + + private final String instantTime; + private final int taskPartitionId; + private final long taskId; + private final long taskEpochId; + private final HoodieTable hoodieTable; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final List writeStatusList = new ArrayList<>(); + + private HoodieRowCreateHandle handle; + private String lastKnownPartitionPath = null; + private String fileIdPrefix; + private int numFilesWritten = 0; + + public BulkInsertDataInternalWriterHelper(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType) { + this.hoodieTable = hoodieTable; + this.writeConfig = writeConfig; + this.instantTime = instantTime; + this.taskPartitionId = taskPartitionId; + this.taskId = taskId; + this.taskEpochId = taskEpochId; + this.structType = structType; + this.fileIdPrefix = UUID.randomUUID().toString(); + } + + 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(); + + if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) { + LOG.info("Creating new file for partition path " + partitionPath); + createNewHandle(partitionPath); + lastKnownPartitionPath = partitionPath; + } + handle.write(record); + } catch (Throwable t) { + LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t); + throw t; + } + } + + public List getWriteStatuses() throws IOException { + close(); + return writeStatusList; + } + + public void abort() { + } + + private void createNewHandle(String partitionPath) throws IOException { + if (null != handle) { + close(); + } + handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(), + instantTime, taskPartitionId, taskId, taskEpochId, structType); + } + + public void close() throws IOException { + if (null != handle) { + writeStatusList.add(handle.close()); + handle = null; + } + } + + private String getNextFileId() { + return String.format("%s-%d", fileIdPrefix, numFilesWritten++); + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java new file mode 100644 index 0000000000000..b40d36bea45e6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; + +import java.util.List; + +/** + * Helper class for HoodieDataSourceInternalWriter used by Spark datasource v2. + */ +public class DataSourceInternalWriterHelper { + + private static final Logger LOG = LogManager.getLogger(DataSourceInternalWriterHelper.class); + public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time"; + + private final String instantTime; + private final HoodieTableMetaClient metaClient; + private final SparkRDDWriteClient writeClient; + private final HoodieTable hoodieTable; + private final WriteOperationType operationType; + + public DataSourceInternalWriterHelper(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession sparkSession, Configuration configuration) { + this.instantTime = instantTime; + this.operationType = WriteOperationType.BULK_INSERT; + this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true); + writeClient.setOperationType(operationType); + writeClient.startCommitWithTime(instantTime); + this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath()); + this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient); + } + + public boolean useCommitCoordinator() { + return true; + } + + public void onDataWriterCommit(String message) { + LOG.info("Received commit of a data writer = " + message); + } + + public void commit(List writeStatList) { + try { + writeClient.commitStats(instantTime, writeStatList, Option.empty(), + DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType())); + } catch (Exception ioe) { + throw new HoodieException(ioe.getMessage(), ioe); + } finally { + writeClient.close(); + } + } + + public void abort() { + LOG.error("Commit " + instantTime + " aborted "); + writeClient.rollback(instantTime); + writeClient.close(); + } + + public void createInflightCommit() { + metaClient.getActiveTimeline().transitionRequestedToInflight( + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); + } + + public HoodieTable getHoodieTable() { + return hoodieTable; + } + + public WriteOperationType getWriteOperationType() { + return operationType; + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java new file mode 100644 index 0000000000000..d66a5ee51a0c6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.testutils.HoodieClientTestHarness; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +import java.util.List; +import java.util.Random; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Base class for TestHoodieBulkInsertDataInternalWriter. + */ +public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarness { + + protected static final Random RANDOM = new Random(); + + @BeforeEach + public void setUp() throws Exception { + initSparkContexts(); + initPath(); + initFileSystem(); + initTestDataGenerator(); + initMetaClient(); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupResources(); + } + + protected void assertWriteStatuses(List writeStatuses, int batches, int size, + Option> fileAbsPaths, Option> fileNames) { + assertEquals(batches, writeStatuses.size()); + int counter = 0; + for (HoodieInternalWriteStatus writeStatus : writeStatuses) { + // verify write status + assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath()); + assertEquals(writeStatus.getTotalRecords(), size); + assertNull(writeStatus.getGlobalError()); + assertEquals(writeStatus.getFailedRowsSize(), 0); + assertEquals(writeStatus.getTotalErrorRecords(), 0); + assertFalse(writeStatus.hasErrors()); + assertNotNull(writeStatus.getFileId()); + String fileId = writeStatus.getFileId(); + if (fileAbsPaths.isPresent()) { + fileAbsPaths.get().add(basePath + "/" + writeStatus.getStat().getPath()); + } + if (fileNames.isPresent()) { + fileNames.get().add(writeStatus.getStat().getPath() + .substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1)); + } + HoodieWriteStat writeStat = writeStatus.getStat(); + assertEquals(size, writeStat.getNumInserts()); + assertEquals(size, writeStat.getNumWrites()); + assertEquals(fileId, writeStat.getFileId()); + assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); + assertEquals(0, writeStat.getNumDeletes()); + assertEquals(0, writeStat.getNumUpdateWrites()); + assertEquals(0, writeStat.getTotalWriteErrors()); + } + } + + protected void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime, Option> fileNames) { + // verify 3 meta fields that are filled in within create handle + actualRows.collectAsList().forEach(entry -> { + assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); + assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); + if (fileNames.isPresent()) { + assertTrue(fileNames.get().contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS + .get(HoodieRecord.FILENAME_METADATA_FIELD)))); + } + assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); + }); + + // after trimming 2 of the meta fields, rest of the fields should match + Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); + assertEquals(0, trimmedActual.except(trimmedExpected).count()); + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java index 025a828dcad17..2ba1e286481cc 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java @@ -193,8 +193,12 @@ public List generateUpdates(Integer n) throws IOException { * @return list of hoodie records to delete */ public List generateDeletes(List rows) { - return rows.stream().map(row -> - convertToString(row.getAs("uuid"), row.getAs("partitionpath"))).filter(os -> os.isPresent()).map(os -> os.get()) + // if row.length() == 2, then the record contains "uuid" and "partitionpath" fields, otherwise, + // another field "ts" is available + return rows.stream().map(row -> row.length() == 2 + ? convertToString(row.getAs("uuid"), row.getAs("partitionpath"), null) : + convertToString(row.getAs("uuid"), row.getAs("partitionpath"), row.getAs("ts")) + ).filter(os -> os.isPresent()).map(os -> os.get()) .collect(Collectors.toList()); } @@ -215,10 +219,10 @@ private static Option convertToString(HoodieRecord record) { } } - private static Option convertToString(String uuid, String partitionPath) { + private static Option convertToString(String uuid, String partitionPath, Long ts) { StringBuffer stringBuffer = new StringBuffer(); stringBuffer.append("{"); - stringBuffer.append("\"ts\": 0.0,"); + stringBuffer.append("\"ts\": \"" + (ts == null ? "0.0" : ts) + "\","); stringBuffer.append("\"uuid\": \"" + uuid + "\","); stringBuffer.append("\"partitionpath\": \"" + partitionPath + "\""); stringBuffer.append("}"); diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala index a1e9947cafacf..f7415f93f6b2d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieBootstrapRelation.scala @@ -91,6 +91,9 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, // Get required schemas for column pruning var requiredDataSchema = StructType(Seq()) var requiredSkeletonSchema = StructType(Seq()) + // requiredColsSchema is the schema of requiredColumns, note that requiredColumns is in a random order + // so requiredColsSchema is not always equal to (requiredSkeletonSchema.fields ++ requiredDataSchema.fields) + var requiredColsSchema = StructType(Seq()) requiredColumns.foreach(col => { var field = dataSchema.find(_.name == col) if (field.isDefined) { @@ -99,6 +102,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, field = skeletonSchema.find(_.name == col) requiredSkeletonSchema = requiredSkeletonSchema.add(field.get) } + requiredColsSchema = requiredColsSchema.add(field.get) }) // Prepare readers for reading data file and skeleton files @@ -129,7 +133,7 @@ class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext, sparkSession = _sqlContext.sparkSession, dataSchema = fullSchema, partitionSchema = StructType(Seq.empty), - requiredSchema = StructType(requiredSkeletonSchema.fields ++ requiredDataSchema.fields), + requiredSchema = requiredColsSchema, filters = filters, options = Map.empty, hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf()) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index d66103600f9a1..988c9f9191e71 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, B import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} -import org.apache.hudi.internal.HoodieDataSourceInternalWriter +import org.apache.hudi.internal.{HoodieDataSourceInternalWriter, DataSourceInternalWriterHelper} import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.log4j.LogManager import org.apache.spark.SPARK_VERSION @@ -130,9 +130,6 @@ private[hudi] object HoodieSparkSqlWriter { // scalastyle:off if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean && operation == WriteOperationType.BULK_INSERT) { - if (!SPARK_VERSION.startsWith("2.")) { - throw new HoodieException("Bulk insert using row writer is not supported with Spark 3. To use row writer please switch to spark 2.") - } val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName, basePath, path, instantTime) return (success, commitTime, common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig) @@ -299,10 +296,22 @@ private[hudi] object HoodieSparkSqlWriter { val nameSpace = s"hoodie.${tblName}" val writeConfig = DataSourceUtils.createHoodieConfig(null, path.get, tblName, mapAsJavaMap(parameters)) val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace) - hoodieDF.write.format("org.apache.hudi.internal") - .option(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY, instantTime) - .options(parameters) - .save() + if (SPARK_VERSION.startsWith("2.")) { + hoodieDF.write.format("org.apache.hudi.internal") + .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) + .options(parameters) + .save() + } else if (SPARK_VERSION.startsWith("3.")) { + hoodieDF.write.format("org.apache.hudi.spark3.internal") + .option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime) + .option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL, hoodieDF.schema.toDDL) + .options(parameters) + .mode(SaveMode.Append) + .save() + } else { + throw new HoodieException("Bulk insert using row writer is not supported with current Spark version." + + " To use row writer please switch to spark 2 or spark 3") + } val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) val metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean) val syncHiveSucess = if (hiveSyncEnabled || metaSyncEnabled) { @@ -364,6 +373,8 @@ private[hudi] object HoodieSparkSqlWriter { hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY) hiveSyncConfig.useJdbc = parameters(HIVE_USE_JDBC_OPT_KEY).toBoolean hiveSyncConfig.supportTimestamp = parameters.get(HIVE_SUPPORT_TIMESTAMP).exists(r => r.toBoolean) + hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY, + DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean hiveSyncConfig } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index dd83bf836433a..51ca72e0411f4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -107,6 +107,7 @@ class TestCOWDataSource extends HoodieClientTestBase { .options(commonOpts) .mode(SaveMode.Append) .save(basePath) + val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) val snapshotDF2 = spark.read.format("hudi").load(basePath + "/*/*/*/*") assertEquals(100, snapshotDF2.count()) @@ -114,7 +115,7 @@ class TestCOWDataSource extends HoodieClientTestBase { // Upsert Operation without Hudi metadata columns val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList - val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2 , 2)) val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() inputDF2.write.format("org.apache.hudi") @@ -122,7 +123,7 @@ class TestCOWDataSource extends HoodieClientTestBase { .mode(SaveMode.Append) .save(basePath) - val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath) + val commitInstantTime3 = HoodieDataSourceHelpers.latestCommit(fs, basePath) assertEquals(3, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size()) // Snapshot Query @@ -154,18 +155,18 @@ class TestCOWDataSource extends HoodieClientTestBase { // pull the latest commit val hoodieIncViewDF2 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2) .load(basePath) assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect() assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + assertEquals(commitInstantTime3, countsPerCommit(0).get(0)) // pull the latest commit within certain partitions val hoodieIncViewDF3 = spark.read.format("org.apache.hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2) .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*") .load(basePath) assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala index f24e5ad5bacc1..2a6a0a71360af 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala @@ -17,9 +17,6 @@ package org.apache.hudi.functional -import java.time.Instant -import java.util.Collections - import collection.JavaConverters._ import org.apache.hadoop.fs.FileSystem import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider @@ -37,10 +34,13 @@ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.io.TempDir +import java.time.Instant +import java.util.Collections + class TestDataSourceForBootstrap { var spark: SparkSession = _ - val commonOpts = Map( + val commonOpts: Map[String, String] = Map( HoodieWriteConfig.INSERT_PARALLELISM -> "4", HoodieWriteConfig.UPSERT_PARALLELISM -> "4", HoodieWriteConfig.DELETE_PARALLELISM -> "4", @@ -56,6 +56,14 @@ class TestDataSourceForBootstrap { var srcPath: String = _ var fs: FileSystem = _ + val partitionPaths: List[String] = List("2020-04-01", "2020-04-02", "2020-04-03") + val numRecords: Int = 100 + val numRecordsUpdate: Int = 10 + val verificationRowKey: String = "trip_0" + val verificationCol: String = "driver" + val originalVerificationVal: String = "driver_0" + val updatedVerificationVal: String = "driver_update" + @BeforeEach def initialize(@TempDir tempDir: java.nio.file.Path) { spark = SparkSession.builder .appName("Hoodie Datasource test") @@ -83,7 +91,6 @@ class TestDataSourceForBootstrap { @Test def testMetadataBootstrapCOWNonPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, Collections.emptyList(), jsc, @@ -96,20 +103,7 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.NonpartitionedKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count var hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") @@ -117,18 +111,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, Collections.emptyList(), jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator") .mode(SaveMode.Append) .save(basePath) @@ -141,36 +131,11 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF1.count()) assertEquals(numRecordsUpdate, hoodieROViewDF1.filter(s"timestamp == $updateTimestamp").count()) - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) - - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) - - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); - - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = false, isHiveStylePartitioned = false) } @Test def testMetadataBootstrapCOWHiveStylePartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -184,20 +149,7 @@ class TestDataSourceForBootstrap { .save(srcPath) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") @@ -205,18 +157,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") // Required because source data is hive style partitioned .option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true") @@ -231,49 +179,14 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF2.count()) assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) - - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) - - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); - - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) - - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/datestr=2020-04-02/*") - .load(basePath) - - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = true) } @Test def testMetadataBootstrapCOWPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) - var sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, + val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, spark.sqlContext) // Writing data for each partition instead of using partitionBy to avoid hive-style partitioning and hence @@ -288,92 +201,56 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") assertEquals(numRecords, hoodieROViewDF1.count()) - // Perform upsert - val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 - var updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, - jsc, spark.sqlContext) - - updateDF.write + // Perform upsert based on the written bootstrap table + val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + updateDf1.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option("hoodie.upsert.shuffle.parallelism", "4") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) - val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + // Read table after upsert and verify the updated value assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) - - // Read table after upsert and verify count val hoodieROViewDF2 = spark.read.format("hudi").load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF2.count()) - assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) - - // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written - // during bootstrap - val hoodieIncViewDF1 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath) + hoodieROViewDF2.collect() + assertEquals(updatedVerificationVal, hoodieROViewDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) - assertEquals(numRecords, hoodieIncViewDF1.count()) - var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) + // Perform upsert based on the source data + val updateTimestamp = Instant.now.toEpochMilli + val updateDF2 = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, + jsc, spark.sqlContext) - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit - val hoodieIncViewDF2 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .load(basePath); + updateDF2.write + .format("hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") + .mode(SaveMode.Append) + .save(basePath) - assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) - countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); - assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + val commitInstantTime3: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*") - .load(basePath) + // Read table after upsert and verify count + val hoodieROViewDF3 = spark.read.format("hudi").load(basePath + "/*") + assertEquals(numRecords, hoodieROViewDF3.count()) + assertEquals(numRecordsUpdate, hoodieROViewDF3.filter(s"timestamp == $updateTimestamp").count()) - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime3, isPartitioned = true, isHiveStylePartitioned = false) } @Test def testMetadataBootstrapMORPartitionedInlineCompactionOn(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -391,21 +268,7 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write - .format("hudi") - .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) - .save(basePath) - - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) // Read bootstrapped table and verify count val hoodieROViewDF1 = spark.read.format("hudi") @@ -416,18 +279,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .option(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true") .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1") @@ -449,8 +308,6 @@ class TestDataSourceForBootstrap { @Test def testMetadataBootstrapMORPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -468,64 +325,64 @@ class TestDataSourceForBootstrap { }) // Perform bootstrap - val bootstrapDF = spark.emptyDataFrame - bootstrapDF.write + val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + + // Read bootstrapped table and verify count + val hoodieROViewDF1 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, + DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) + .load(basePath + "/*") + assertEquals(numRecords, hoodieROViewDF1.count()) + + // Perform upsert based on the written bootstrap table + val updateDf1 = hoodieROViewDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal)) + updateDf1.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") - .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, "org.apache.hudi.keygen.SimpleKeyGenerator") - .mode(SaveMode.Overwrite) + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") + .mode(SaveMode.Append) .save(basePath) - val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) - assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) - - // Read bootstrapped table and verify count - val hoodieROViewDF1 = spark.read.format("hudi") + // Read table after upsert and verify the value + assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) + val hoodieROViewDF2 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF1.count()) + hoodieROViewDF2.collect() + assertEquals(originalVerificationVal, hoodieROViewDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0)) - // Perform upsert + // Perform upsert based on the source data val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 - val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, + val updateDF2 = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) - updateDF.write + updateDF2.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) - // Expect 1 new commit since meta bootstrap - delta commit (because inline compaction is off) - assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) + // Expect 2 new commit since meta bootstrap - 2 delta commits (because inline compaction is off) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size()) // Read table after upsert and verify count. Since we have inline compaction off the RO view will have // no updated rows. - val hoodieROViewDF2 = spark.read.format("hudi") + val hoodieROViewDF3 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL) .load(basePath + "/*") - assertEquals(numRecords, hoodieROViewDF2.count()) - assertEquals(0, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) + assertEquals(numRecords, hoodieROViewDF3.count()) + assertEquals(0, hoodieROViewDF3.filter(s"timestamp == $updateTimestamp").count()) } @Test def testFullBootstrapCOWPartitioned(): Unit = { val timestamp = Instant.now.toEpochMilli - val numRecords = 100 - val partitionPaths = List("2020-04-01", "2020-04-02", "2020-04-03") val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext) val sourceDF = TestBootstrap.generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths.asJava, jsc, @@ -547,11 +404,8 @@ class TestDataSourceForBootstrap { bootstrapDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName) .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, classOf[FullRecordBootstrapModeSelector].getName) @@ -568,18 +422,14 @@ class TestDataSourceForBootstrap { // Perform upsert val updateTimestamp = Instant.now.toEpochMilli - val numRecordsUpdate = 10 val updateDF = TestBootstrap.generateTestRawTripDataset(updateTimestamp, 0, numRecordsUpdate, partitionPaths.asJava, jsc, spark.sqlContext) updateDF.write .format("hudi") .options(commonOpts) - .option(HoodieWriteConfig.TABLE_NAME, "hoodie_test") .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "timestamp") .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr") .mode(SaveMode.Append) .save(basePath) @@ -592,39 +442,64 @@ class TestDataSourceForBootstrap { assertEquals(numRecords, hoodieROViewDF2.count()) assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count()) + verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = false) + } + + def runMetadataBootstrapAndVerifyCommit(tableType: String): String = { + val bootstrapDF = spark.emptyDataFrame + bootstrapDF.write + .format("hudi") + .options(commonOpts) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName) + .mode(SaveMode.Overwrite) + .save(basePath) + + val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath) + assertEquals(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, commitInstantTime1) + commitInstantTime1 + } + + def verifyIncrementalViewResult(bootstrapCommitInstantTime: String, latestCommitInstantTime: String, + isPartitioned: Boolean, isHiveStylePartitioned: Boolean): Unit = { // incrementally pull only changes in the bootstrap commit, which would pull all the initial records written // during bootstrap val hoodieIncViewDF1 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000") - .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) .load(basePath) assertEquals(numRecords, hoodieIncViewDF1.count()) var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect(); assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime1, countsPerCommit(0).get(0)) + assertEquals(bootstrapCommitInstantTime, countsPerCommit(0).get(0)) - // incrementally pull only changes in the latest commit, which would pull only the updated records in the - // latest commit + // incrementally pull only changes after bootstrap commit, which would pull only the updated records in the + // later commits val hoodieIncViewDF2 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) .load(basePath); assertEquals(numRecordsUpdate, hoodieIncViewDF2.count()) countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect(); assertEquals(1, countsPerCommit.length) - assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) - - // pull the latest commit within certain partitions - val hoodieIncViewDF3 = spark.read.format("hudi") - .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) - .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1) - .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2020-04-02/*") - .load(basePath) - - assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), - hoodieIncViewDF3.count()) + assertEquals(latestCommitInstantTime, countsPerCommit(0).get(0)) + + if (isPartitioned) { + val relativePartitionPath = if (isHiveStylePartitioned) "/datestr=2020-04-02/*" else "/2020-04-02/*" + // pull the update commits within certain partitions + val hoodieIncViewDF3 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime) + .option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, relativePartitionPath) + .load(basePath) + + assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(), + hoodieIncViewDF3.count()) + } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 7a902c14b5f23..b07f00f61bf5f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -18,16 +18,20 @@ package org.apache.hudi.functional import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hudi.common.model.FileSlice import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestTable} import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings -import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.config.{HoodieClusteringConfig, HoodieStorageConfig, HoodieWriteConfig} import org.apache.hudi.exception.TableNotFoundException import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.log4j.LogManager import org.apache.spark.sql._ import org.apache.spark.sql.streaming.{OutputMode, Trigger} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.apache.spark.sql.types.StructType +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} +import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import scala.collection.JavaConversions._ @@ -64,35 +68,19 @@ class TestStructuredStreaming extends HoodieClientTestBase { cleanupFileSystem() } - @Test - def testStructuredStreaming(): Unit = { - fs.delete(new Path(basePath), true) - val sourcePath = basePath + "/source" - val destPath = basePath + "/dest" - fs.mkdirs(new Path(sourcePath)) - - // First chunk of data - val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList - val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) - - // Second chunk of data - val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList - val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) - val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() - + def initStreamingWriteFuture(schema: StructType, sourcePath: String, destPath: String, hudiOptions: Map[String, String]): Future[Unit] = { // define the source of streaming val streamingInput = spark.readStream - .schema(inputDF1.schema) + .schema(schema) .json(sourcePath) - - val f1 = Future { + Future { println("streaming starting") //'writeStream' can be called only on streaming Dataset/DataFrame streamingInput .writeStream .format("org.apache.hudi") - .options(commonOpts) + .options(hudiOptions) .trigger(Trigger.ProcessingTime(100)) .option("checkpointLocation", basePath + "/checkpoint") .outputMode(OutputMode.Append) @@ -100,6 +88,29 @@ class TestStructuredStreaming extends HoodieClientTestBase { .awaitTermination(10000) println("streaming ends") } + } + + def initStreamingSourceAndDestPath(sourceDirName: String, destDirName: String): (String, String) = { + fs.delete(new Path(basePath), true) + val sourcePath = basePath + "/" + sourceDirName + val destPath = basePath + "/" + destDirName + fs.mkdirs(new Path(sourcePath)) + (sourcePath, destPath) + } + + @Test + def testStructuredStreaming(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + // First chunk of data + val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + // Second chunk of data + val records2 = recordsToStrings(dataGen.generateUpdates("001", 100)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count() + + val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, commonOpts) val f2 = Future { inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) @@ -113,7 +124,7 @@ class TestStructuredStreaming extends HoodieClientTestBase { assert(hoodieROViewDF1.count() == 100) inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) - // wait for spark streaming to process one microbatch + // wait for spark streaming to process second microbatch waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5) val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, destPath) assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size()) @@ -177,4 +188,112 @@ class TestStructuredStreaming extends HoodieClientTestBase { if (!success) throw new IllegalStateException("Timed-out waiting for " + numCommits + " commits to appear in " + tablePath) numInstants } + + def getInlineClusteringOpts( isInlineClustering: String, clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = { + commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING_PROP -> isInlineClustering, + HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP -> clusteringNumCommit, + HoodieStorageConfig.PARQUET_FILE_MAX_BYTES -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString + ) + } + + @Test + def testStructuredStreamingWithInlineClustering(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + + def checkClusteringResult(destPath: String):Unit = { + // check have schedule clustering and clustering file group to one + waitTillHasCompletedReplaceInstant(destPath, 120, 5) + metaClient.reloadActiveTimeline() + assertEquals(1, getLatestFileGroupsFileId.size) + } + structuredStreamingForTestClusteringRunner(sourcePath, destPath, true, checkClusteringResult) + } + + @Test + def testStructuredStreamingWithoutInlineClustering(): Unit = { + val (sourcePath, destPath) = initStreamingSourceAndDestPath("source", "dest") + + def checkClusteringResult(destPath: String):Unit = { + val msg = "Should have replace commit completed" + assertThrows(classOf[IllegalStateException], new Executable { + override def execute(): Unit = { + waitTillHasCompletedReplaceInstant(destPath, 120, 5) + } + } + , "Should have replace commit completed") + println(msg) + } + structuredStreamingForTestClusteringRunner(sourcePath, destPath, false, checkClusteringResult) + } + + def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String, + isInlineClustering: Boolean, checkClusteringResult: String => Unit): Unit = { + // First insert of data + val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList + val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + + // Second insert of data + val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList + val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + + val hudiOptions = getInlineClusteringOpts(isInlineClustering.toString, "2", 100) + val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, hudiOptions) + + val f2 = Future { + inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process one microbatch + val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000")) + + inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) + // wait for spark streaming to process second microbatch + waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5) + assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size()) + + // check have more than one file group + this.metaClient = new HoodieTableMetaClient(fs.getConf, destPath, true) + assertTrue(getLatestFileGroupsFileId().size > 1) + + // check clustering result + checkClusteringResult(destPath) + + // check data correct after clustering + val hoodieROViewDF2 = spark.read.format("org.apache.hudi") + .load(destPath + "/*/*/*/*") + assertEquals(200, hoodieROViewDF2.count()) + } + Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf) + } + + private def getLatestFileGroupsFileId():Array[String] = { + getHoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline, + HoodieTestTable.of(metaClient).listAllBaseFiles()) + tableView.getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) + .toArray().map(slice => slice.asInstanceOf[FileSlice].getFileGroupId.getFileId) + } + + @throws[InterruptedException] + private def waitTillHasCompletedReplaceInstant(tablePath: String, + timeoutSecs: Int, sleepSecsAfterEachRun: Int) = { + val beginTime = System.currentTimeMillis + var currTime = beginTime + val timeoutMsecs = timeoutSecs * 1000 + var success = false + while ({!success && (currTime - beginTime) < timeoutMsecs}) try { + this.metaClient.reloadActiveTimeline() + val completeReplaceSize = this.metaClient.getActiveTimeline.getCompletedReplaceTimeline().getInstants.toArray.size + println("completeReplaceSize:" + completeReplaceSize) + if(completeReplaceSize > 0) { + success = true + } + } catch { + case te: TableNotFoundException => + log.info("Got table not found exception. Retrying") + } finally { + Thread.sleep(sleepSecsAfterEachRun * 1000) + currTime = System.currentTimeMillis + } + if (!success) throw new IllegalStateException("Timed-out waiting for " + " have completed replace instant appear in " + tablePath) + } + } diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml index 8947bb79f6d74..cd44e79ceecde 100644 --- a/hudi-spark-datasource/hudi-spark2/pom.xml +++ b/hudi-spark-datasource/hudi-spark2/pom.xml @@ -214,6 +214,14 @@ test-jar test + + org.apache.hudi + hudi-spark-common + ${project.version} + tests + test-jar + test + org.junit.jupiter diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java index 5fb71df777df4..526f0ce4783d6 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -21,11 +21,7 @@ import org.apache.hudi.DataSourceUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.sources.DataSourceRegister; import org.apache.spark.sql.sources.v2.DataSourceOptions; import org.apache.spark.sql.sources.v2.DataSourceV2; @@ -40,14 +36,8 @@ /** * DataSource V2 implementation for managing internal write logic. Only called internally. */ -public class DefaultSource implements DataSourceV2, ReadSupport, WriteSupport, - DataSourceRegister { - - private static final Logger LOG = LogManager - .getLogger(DefaultSource.class); - - private SparkSession sparkSession = null; - private Configuration configuration = null; +public class DefaultSource extends BaseDefaultSource implements DataSourceV2, + ReadSupport, WriteSupport, DataSourceRegister { @Override public String shortName() { @@ -67,25 +57,11 @@ public DataSourceReader createReader(DataSourceOptions options) { @Override public Optional createWriter(String writeUUID, StructType schema, SaveMode mode, DataSourceOptions options) { - String instantTime = options.get(HoodieDataSourceInternalWriter.INSTANT_TIME_OPT_KEY).get(); + String instantTime = options.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY).get(); String path = options.get("path").get(); String tblName = options.get(HoodieWriteConfig.TABLE_NAME).get(); HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, options.asMap()); return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(), getConfiguration())); } - - private SparkSession getSparkSession() { - if (sparkSession == null) { - sparkSession = SparkSession.builder().getOrCreate(); - } - return sparkSession; - } - - private Configuration getConfiguration() { - if (configuration == null) { - this.configuration = getSparkSession().sparkContext().hadoopConfiguration(); - } - return configuration; - } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java index 7aa0fc6a3846f..3ce8d776a8e68 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieBulkInsertDataInternalWriter.java @@ -18,102 +18,42 @@ package org.apache.hudi.internal; -import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.io.HoodieRowCreateHandle; import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; import org.apache.spark.sql.types.StructType; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; /** * Hoodie's Implementation of {@link DataWriter}. This is used in data source implementation for bulk insert. */ public class HoodieBulkInsertDataInternalWriter implements DataWriter { - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(HoodieBulkInsertDataInternalWriter.class); - - private final String instantTime; - private final int taskPartitionId; - private final long taskId; - private final long taskEpochId; - private final HoodieTable hoodieTable; - private final HoodieWriteConfig writeConfig; - private final StructType structType; - private final List writeStatusList = new ArrayList<>(); - - private HoodieRowCreateHandle handle; - private String lastKnownPartitionPath = null; - private String fileIdPrefix = null; - private int numFilesWritten = 0; + private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper; public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, String instantTime, int taskPartitionId, long taskId, long taskEpochId, StructType structType) { - this.hoodieTable = hoodieTable; - this.writeConfig = writeConfig; - this.instantTime = instantTime; - this.taskPartitionId = taskPartitionId; - this.taskId = taskId; - this.taskEpochId = taskEpochId; - this.structType = structType; - this.fileIdPrefix = UUID.randomUUID().toString(); + this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable, + writeConfig, instantTime, taskPartitionId, taskId, taskEpochId, structType); } @Override 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(); - - if ((lastKnownPartitionPath == null) || !lastKnownPartitionPath.equals(partitionPath) || !handle.canWrite()) { - LOG.info("Creating new file for partition path " + partitionPath); - createNewHandle(partitionPath); - lastKnownPartitionPath = partitionPath; - } - handle.write(record); - } catch (Throwable t) { - LOG.error("Global error thrown while trying to write records in HoodieRowCreateHandle ", t); - throw t; - } + bulkInsertWriterHelper.write(record); } @Override public WriterCommitMessage commit() throws IOException { - close(); - return new HoodieWriterCommitMessage(writeStatusList); + return new HoodieWriterCommitMessage(bulkInsertWriterHelper.getWriteStatuses()); } @Override - public void abort() throws IOException { - } - - private void createNewHandle(String partitionPath) throws IOException { - if (null != handle) { - close(); - } - handle = new HoodieRowCreateHandle(hoodieTable, writeConfig, partitionPath, getNextFileId(), - instantTime, taskPartitionId, taskId, taskEpochId, structType); - } - - public void close() throws IOException { - if (null != handle) { - writeStatusList.add(handle.close()); - } - } - - protected String getNextFileId() { - return String.format("%s-%d", fileIdPrefix, numFilesWritten++); + public void abort() { + bulkInsertWriterHelper.abort(); } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index e8cbff80a2c2e..4b3dafc6264f7 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -18,24 +18,12 @@ package org.apache.hudi.internal; -import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.DataSourceUtils; -import org.apache.hudi.client.SparkRDDWriteClient; -import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.table.HoodieSparkTable; -import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaSparkContext; + +import org.apache.hadoop.conf.Configuration; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; @@ -53,71 +41,50 @@ */ public class HoodieDataSourceInternalWriter implements DataSourceWriter { - private static final long serialVersionUID = 1L; - private static final Logger LOG = LogManager.getLogger(HoodieDataSourceInternalWriter.class); - public static final String INSTANT_TIME_OPT_KEY = "hoodie.instant.time"; - private final String instantTime; - private final HoodieTableMetaClient metaClient; private final HoodieWriteConfig writeConfig; private final StructType structType; - private final SparkRDDWriteClient writeClient; - private final HoodieTable hoodieTable; - private final WriteOperationType operationType; + private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper; public HoodieDataSourceInternalWriter(String instantTime, HoodieWriteConfig writeConfig, StructType structType, SparkSession sparkSession, Configuration configuration) { this.instantTime = instantTime; this.writeConfig = writeConfig; this.structType = structType; - this.operationType = WriteOperationType.BULK_INSERT; - this.writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true); - writeClient.setOperationType(operationType); - writeClient.startCommitWithTime(instantTime); - this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath()); - this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient); + this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, + sparkSession, configuration); } @Override public DataWriterFactory createWriterFactory() { - metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, instantTime), Option.empty()); - if (WriteOperationType.BULK_INSERT == operationType) { - return new HoodieBulkInsertDataInternalWriterFactory(hoodieTable, writeConfig, instantTime, structType); + dataSourceInternalWriterHelper.createInflightCommit(); + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { + return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), + writeConfig, instantTime, structType); } else { - throw new IllegalArgumentException("Write Operation Type + " + operationType + " not supported "); + throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported "); } } @Override public boolean useCommitCoordinator() { - return true; + return dataSourceInternalWriterHelper.useCommitCoordinator(); } @Override public void onDataWriterCommit(WriterCommitMessage message) { - LOG.info("Received commit of a data writer =" + message); + dataSourceInternalWriterHelper.onDataWriterCommit(message.toString()); } @Override public void commit(WriterCommitMessage[] messages) { List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) - .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList()); - - try { - writeClient.commitStats(instantTime, writeStatList, Option.empty(), - DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType())); - } catch (Exception ioe) { - throw new HoodieException(ioe.getMessage(), ioe); - } finally { - writeClient.close(); - } + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); } @Override public void abort(WriterCommitMessage[] messages) { - LOG.error("Commit " + instantTime + " aborted "); - writeClient.rollback(instantTime); - writeClient.close(); + dataSourceInternalWriterHelper.abort(); } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java index 757000c57c1dd..240e4b9819413 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/HoodieWriterCommitMessage.java @@ -18,28 +18,18 @@ package org.apache.hudi.internal; -import java.util.ArrayList; -import java.util.List; import org.apache.hudi.client.HoodieInternalWriteStatus; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import java.util.List; + /** * Hoodie's {@link WriterCommitMessage} used in datasource implementation. */ -public class HoodieWriterCommitMessage implements WriterCommitMessage { - - private List writeStatuses = new ArrayList<>(); +public class HoodieWriterCommitMessage extends BaseWriterCommitMessage + implements WriterCommitMessage { public HoodieWriterCommitMessage(List writeStatuses) { - this.writeStatuses = writeStatuses; - } - - public List getWriteStatuses() { - return writeStatuses; - } - - @Override - public String toString() { - return "HoodieWriterCommitMessage{" + "writeStatuses=" + writeStatuses + '}'; + super(writeStatuses); } } diff --git a/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java index ac69af51eb8b1..0b021abeb3b5f 100644 --- a/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -18,26 +18,19 @@ package org.apache.hudi.internal; -import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.testutils.HoodieClientTestHarness; -import org.apache.spark.package$; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; -import java.util.Random; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; @@ -45,36 +38,13 @@ import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Unit tests {@link HoodieBulkInsertDataInternalWriter}. */ -public class TestHoodieBulkInsertDataInternalWriter extends HoodieClientTestHarness { - - private static final Random RANDOM = new Random(); - - @BeforeEach - public void setUp() throws Exception { - // this test is only compatible with spark 2 - assumeTrue(package$.MODULE$.SPARK_VERSION().startsWith("2.")); - initSparkContexts("TestHoodieBulkInsertDataInternalWriter"); - initPath(); - initFileSystem(); - initTestDataGenerator(); - initMetaClient(); - } - - @AfterEach - public void tearDown() throws Exception { - cleanupResources(); - } +public class TestHoodieBulkInsertDataInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { @Test public void testDataInternalWriter() throws Exception { @@ -103,15 +73,15 @@ public void testDataInternalWriter() throws Exception { } } - HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); - List fileAbsPaths = new ArrayList<>(); - List fileNames = new ArrayList<>(); + BaseWriterCommitMessage commitMetadata = (BaseWriterCommitMessage) writer.commit(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); // verify write statuses assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames); // verify rows - Dataset result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); assertOutput(totalInputRows, result, instantTime, fileNames); } } @@ -156,15 +126,15 @@ public void testGlobalFailure() throws Exception { // expected } - HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + BaseWriterCommitMessage commitMetadata = (BaseWriterCommitMessage) writer.commit(); - List fileAbsPaths = new ArrayList<>(); - List fileNames = new ArrayList<>(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); // verify write statuses assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames); // verify rows - Dataset result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0])); + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); assertOutput(inputRows, result, instantTime, fileNames); } @@ -176,43 +146,4 @@ private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWrite writer.write(internalRow); } } - - private void assertWriteStatuses(List writeStatuses, int batches, int size, List fileAbsPaths, List fileNames) { - assertEquals(batches, writeStatuses.size()); - int counter = 0; - for (HoodieInternalWriteStatus writeStatus : writeStatuses) { - // verify write status - assertEquals(writeStatus.getTotalRecords(), size); - assertNull(writeStatus.getGlobalError()); - assertEquals(writeStatus.getFailedRowsSize(), 0); - assertNotNull(writeStatus.getFileId()); - String fileId = writeStatus.getFileId(); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3], writeStatus.getPartitionPath()); - fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath()); - fileNames.add(writeStatus.getStat().getPath().substring(writeStatus.getStat().getPath().lastIndexOf('/') + 1)); - HoodieWriteStat writeStat = writeStatus.getStat(); - assertEquals(size, writeStat.getNumInserts()); - assertEquals(size, writeStat.getNumWrites()); - assertEquals(fileId, writeStat.getFileId()); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); - assertEquals(0, writeStat.getNumDeletes()); - assertEquals(0, writeStat.getNumUpdateWrites()); - assertEquals(0, writeStat.getTotalWriteErrors()); - } - } - - private void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime, List fileNames) { - // verify 3 meta fields that are filled in within create handle - actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); - assertTrue(fileNames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)))); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); - }); - - // after trimming 2 of the meta fields, rest of the fields should match - Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - assertEquals(0, trimmedActual.except(trimmedExpected).count()); - } } diff --git a/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java index 454c74d967cd5..184ff771cef4a 100644 --- a/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java +++ b/hudi-spark-datasource/hudi-spark2/src/test/java/org/apache/hudi/internal/TestHoodieDataSourceInternalWriter.java @@ -18,61 +18,32 @@ package org.apache.hudi.internal; -import org.apache.hudi.client.HoodieInternalWriteStatus; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.hudi.testutils.HoodieClientTestUtils; -import org.apache.spark.package$; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assumptions.assumeTrue; /** * Unit tests {@link HoodieDataSourceInternalWriter}. */ -public class TestHoodieDataSourceInternalWriter extends HoodieClientTestHarness { - - private static final Random RANDOM = new Random(); - - @BeforeEach - public void setUp() throws Exception { - // this test is only compatible with spark 2 - assumeTrue(package$.MODULE$.SPARK_VERSION().startsWith("2.")); - initSparkContexts("TestHoodieDataSourceInternalWriter"); - initPath(); - initFileSystem(); - initTestDataGenerator(); - initMetaClient(); - } - - @AfterEach - public void tearDown() throws Exception { - cleanupResources(); - } +public class TestHoodieDataSourceInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { @Test public void testDataSourceWriter() throws Exception { @@ -84,7 +55,7 @@ public void testDataSourceWriter() throws Exception { new HoodieDataSourceInternalWriter(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); DataWriter writer = dataSourceInternalWriter.createWriterFactory().createDataWriter(0, RANDOM.nextLong(), RANDOM.nextLong()); - List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS; List partitionPathsAbs = new ArrayList<>(); for (String partitionPath : partitionPaths) { partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); @@ -112,8 +83,8 @@ public void testDataSourceWriter() throws Exception { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } @Test @@ -155,8 +126,8 @@ public void testMultipleDataSourceWrites() throws Exception { Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } } @@ -199,8 +170,8 @@ public void testLargeWrites() throws Exception { Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); // verify output - assertOutput(totalInputRows, result, instantTime); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); } } @@ -250,8 +221,8 @@ public void testAbort() throws Exception { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify rows - assertOutput(totalInputRows, result, instantTime0); - assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size); + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); // 2nd batch. abort in the end String instantTime1 = "00" + 1; @@ -274,7 +245,7 @@ public void testAbort() throws Exception { result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); // verify rows // only rows from first batch should be present - assertOutput(totalInputRows, result, instantTime0); + assertOutput(totalInputRows, result, instantTime0, Option.empty()); } private void writeRows(Dataset inputRows, DataWriter writer) throws Exception { @@ -284,41 +255,4 @@ private void writeRows(Dataset inputRows, DataWriter writer) t writer.write(internalRow); } } - - private void assertWriteStatuses(List writeStatuses, int batches, int size) { - assertEquals(batches, writeStatuses.size()); - int counter = 0; - for (HoodieInternalWriteStatus writeStatus : writeStatuses) { - assertEquals(writeStatus.getPartitionPath(), HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter % 3]); - assertEquals(writeStatus.getTotalRecords(), size); - assertEquals(writeStatus.getFailedRowsSize(), 0); - assertEquals(writeStatus.getTotalErrorRecords(), 0); - assertFalse(writeStatus.hasErrors()); - assertNull(writeStatus.getGlobalError()); - assertNotNull(writeStatus.getFileId()); - String fileId = writeStatus.getFileId(); - HoodieWriteStat writeStat = writeStatus.getStat(); - assertEquals(size, writeStat.getNumInserts()); - assertEquals(size, writeStat.getNumWrites()); - assertEquals(fileId, writeStat.getFileId()); - assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[counter++ % 3], writeStat.getPartitionPath()); - assertEquals(0, writeStat.getNumDeletes()); - assertEquals(0, writeStat.getNumUpdateWrites()); - assertEquals(0, writeStat.getTotalWriteErrors()); - } - } - - private void assertOutput(Dataset expectedRows, Dataset actualRows, String instantTime) { - // verify 3 meta fields that are filled in within create handle - actualRows.collectAsList().forEach(entry -> { - assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD))); - assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD))); - }); - - // after trimming 2 of the meta fields, rest of the fields should match - Dataset trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - Dataset trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD); - assertEquals(0, trimmedActual.except(trimmedExpected).count()); - } } diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml index 7b1ffa9776517..df914098ba7e1 100644 --- a/hudi-spark-datasource/hudi-spark3/pom.xml +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -153,10 +153,71 @@ true + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.spark3.version} + + + com.fasterxml.jackson.core + jackson-annotations + ${fasterxml.spark3.version} + + + com.fasterxml.jackson.core + jackson-core + ${fasterxml.spark3.version} + + + + org.apache.hudi + hudi-spark-client + ${project.version} + + + org.apache.hudi + hudi-spark-common + ${project.version} + + + + + org.apache.hudi + hudi-client-common + ${project.version} + tests + test-jar + test + org.apache.hudi hudi-spark-client ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-common + ${project.version} + tests + test-jar + test + + + org.apache.hudi + hudi-spark-common + ${project.version} + tests + test-jar + test + + + + org.junit.jupiter + junit-jupiter-api + test diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java new file mode 100644 index 0000000000000..d59b5ad5c2935 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.BaseDefaultSource; +import org.apache.hudi.internal.DataSourceInternalWriterHelper; + +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +import java.util.Map; + +/** + * DataSource V2 implementation for managing internal write logic. Only called internally. + * This class is only compatible with datasource V2 API in Spark 3. + */ +public class DefaultSource extends BaseDefaultSource implements TableProvider { + + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + return StructType.fromDDL(options.get(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL)); + } + + @Override + public Table getTable(StructType schema, Transform[] partitioning, Map properties) { + String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY); + String path = properties.get("path"); + String tblName = properties.get(HoodieWriteConfig.TABLE_NAME); + HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(null, path, tblName, properties); + return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(), + getConfiguration()); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..f67187c2718a8 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriter.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.BulkInsertDataInternalWriterHelper; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; + +import java.io.IOException; + +/** + * Hoodie's Implementation of {@link DataWriter}. This is used in data source "hudi.spark3.internal" implementation for bulk insert. + */ +public class HoodieBulkInsertDataInternalWriter implements DataWriter { + + private final BulkInsertDataInternalWriterHelper bulkInsertWriterHelper; + + public HoodieBulkInsertDataInternalWriter(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, int taskPartitionId, long taskId, StructType structType) { + this.bulkInsertWriterHelper = new BulkInsertDataInternalWriterHelper(hoodieTable, + writeConfig, instantTime, taskPartitionId, taskId, 0, structType); + } + + @Override + public void write(InternalRow record) throws IOException { + bulkInsertWriterHelper.write(record); + } + + @Override + public WriterCommitMessage commit() throws IOException { + return new HoodieWriterCommitMessage(bulkInsertWriterHelper.getWriteStatuses()); + } + + @Override + public void abort() { + bulkInsertWriterHelper.abort(); + } + + @Override + public void close() throws IOException { + bulkInsertWriterHelper.close(); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java new file mode 100644 index 0000000000000..31b43ea7d45d6 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieBulkInsertDataInternalWriterFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.types.StructType; + +/** + * Factory to assist in instantiating {@link HoodieBulkInsertDataInternalWriter}. + */ +public class HoodieBulkInsertDataInternalWriterFactory implements DataWriterFactory { + + private final String instantTime; + private final HoodieTable hoodieTable; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + + public HoodieBulkInsertDataInternalWriterFactory(HoodieTable hoodieTable, HoodieWriteConfig writeConfig, + String instantTime, StructType structType) { + this.hoodieTable = hoodieTable; + this.writeConfig = writeConfig; + this.instantTime = instantTime; + this.structType = structType; + } + + @Override + public DataWriter createWriter(int partitionId, long taskId) { + return new HoodieBulkInsertDataInternalWriter(hoodieTable, writeConfig, instantTime, partitionId, taskId, + structType); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java new file mode 100644 index 0000000000000..b0945156d703d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWrite.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.DataSourceInternalWriterHelper; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; +import org.apache.spark.sql.types.StructType; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Implementation of {@link BatchWrite} for datasource "hudi.spark3.internal" to be used in datasource implementation + * of bulk insert. + */ +public class HoodieDataSourceInternalBatchWrite implements BatchWrite { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final DataSourceInternalWriterHelper dataSourceInternalWriterHelper; + + public HoodieDataSourceInternalBatchWrite(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.dataSourceInternalWriterHelper = new DataSourceInternalWriterHelper(instantTime, writeConfig, structType, + jss, hadoopConfiguration); + } + + @Override + public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { + dataSourceInternalWriterHelper.createInflightCommit(); + if (WriteOperationType.BULK_INSERT == dataSourceInternalWriterHelper.getWriteOperationType()) { + return new HoodieBulkInsertDataInternalWriterFactory(dataSourceInternalWriterHelper.getHoodieTable(), + writeConfig, instantTime, structType); + } else { + throw new IllegalArgumentException("Write Operation Type + " + dataSourceInternalWriterHelper.getWriteOperationType() + " not supported "); + } + } + + @Override + public boolean useCommitCoordinator() { + return dataSourceInternalWriterHelper.useCommitCoordinator(); + } + + @Override + public void onDataWriterCommit(WriterCommitMessage message) { + dataSourceInternalWriterHelper.onDataWriterCommit(message.toString()); + } + + @Override + public void commit(WriterCommitMessage[] messages) { + List writeStatList = Arrays.stream(messages).map(m -> (HoodieWriterCommitMessage) m) + .flatMap(m -> m.getWriteStatuses().stream().map(HoodieInternalWriteStatus::getStat)).collect(Collectors.toList()); + dataSourceInternalWriterHelper.commit(writeStatList); + } + + @Override + public void abort(WriterCommitMessage[] messages) { + dataSourceInternalWriterHelper.abort(); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java new file mode 100644 index 0000000000000..10e2e64f11387 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalBatchWriteBuilder.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.hadoop.conf.Configuration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +/** + * Implementation of {@link WriteBuilder} for datasource "hudi.spark3.internal" to be used in datasource implementation + * of bulk insert. + */ +public class HoodieDataSourceInternalBatchWriteBuilder implements WriteBuilder { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final SparkSession jss; + private final Configuration hadoopConfiguration; + + public HoodieDataSourceInternalBatchWriteBuilder(String instantTime, HoodieWriteConfig writeConfig, StructType structType, + SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = writeConfig; + this.structType = structType; + this.jss = jss; + this.hadoopConfiguration = hadoopConfiguration; + } + + @Override + public BatchWrite buildForBatch() { + return new HoodieDataSourceInternalBatchWrite(instantTime, writeConfig, structType, jss, + hadoopConfiguration); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java new file mode 100644 index 0000000000000..f1fded033dcfc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieDataSourceInternalTable.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.SupportsWrite; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +import java.util.HashSet; +import java.util.Set; + +/** + * Hoodie's Implementation of {@link SupportsWrite}. This is used in data source "hudi.spark3.internal" implementation for bulk insert. + */ +class HoodieDataSourceInternalTable implements SupportsWrite { + + private final String instantTime; + private final HoodieWriteConfig writeConfig; + private final StructType structType; + private final SparkSession jss; + private final Configuration hadoopConfiguration; + + public HoodieDataSourceInternalTable(String instantTime, HoodieWriteConfig config, + StructType schema, SparkSession jss, Configuration hadoopConfiguration) { + this.instantTime = instantTime; + this.writeConfig = config; + this.structType = schema; + this.jss = jss; + this.hadoopConfiguration = hadoopConfiguration; + } + + @Override + public String name() { + return this.getClass().toString(); + } + + @Override + public StructType schema() { + return structType; + } + + @Override + public Set capabilities() { + return new HashSet() {{ + add(TableCapability.BATCH_WRITE); + add(TableCapability.TRUNCATE); + }}; + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo logicalWriteInfo) { + return new HoodieDataSourceInternalBatchWriteBuilder(instantTime, writeConfig, structType, jss, + hadoopConfiguration); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java new file mode 100644 index 0000000000000..7fe787deb8a08 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/java/org/apache/hudi/spark3/internal/HoodieWriterCommitMessage.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.client.HoodieInternalWriteStatus; +import org.apache.hudi.internal.BaseWriterCommitMessage; +import org.apache.spark.sql.connector.write.WriterCommitMessage; + +import java.util.List; + +/** + * Hoodie's {@link WriterCommitMessage} used in datasource "hudi.spark3.internal" implementation. + */ +public class HoodieWriterCommitMessage extends BaseWriterCommitMessage + implements WriterCommitMessage { + + public HoodieWriterCommitMessage(List writeStatuses) { + super(writeStatuses); + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java new file mode 100644 index 0000000000000..ffb649bd3970a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieBulkInsertDataInternalWriter.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.HoodieBulkInsertInternalWriterTestBase; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; +import static org.junit.jupiter.api.Assertions.fail; + +/** + * Unit tests {@link HoodieBulkInsertDataInternalWriter}. + */ +public class TestHoodieBulkInsertDataInternalWriter extends + HoodieBulkInsertInternalWriterTestBase { + + @Test + public void testDataInternalWriter() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + // execute N rounds + for (int i = 0; i < 5; i++) { + String instantTime = "00" + i; + // init writer + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE); + + int size = 10 + RANDOM.nextInt(1000); + // write N rows to partition1, N rows to partition2 and N rows to partition3 ... Each batch should create a new RowCreateHandle and a new file + int batches = 5; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); + + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), batches, size, fileAbsPaths, fileNames); + + // verify rows + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); + assertOutput(totalInputRows, result, instantTime, fileNames); + } + } + + + /** + * Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch2 of invalid records which is expected + * to throw Global Error. Verify global error is set appropriately and only first batch of records are written to disk. + */ + @Test + public void testGlobalFailure() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0]; + + String instantTime = "001"; + HoodieBulkInsertDataInternalWriter writer = new HoodieBulkInsertDataInternalWriter(table, cfg, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), STRUCT_TYPE); + + int size = 10 + RANDOM.nextInt(100); + int totalFailures = 5; + // Generate first batch of valid rows + Dataset inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false); + List internalRows = toInternalRows(inputRows, ENCODER); + + // generate some failures rows + for (int i = 0; i < totalFailures; i++) { + internalRows.add(getInternalRowWithError(partitionPath)); + } + + // generate 2nd batch of valid rows + Dataset inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false); + internalRows.addAll(toInternalRows(inputRows2, ENCODER)); + + // issue writes + try { + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + fail("Should have failed"); + } catch (Throwable e) { + // expected + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + + Option> fileAbsPaths = Option.of(new ArrayList<>()); + Option> fileNames = Option.of(new ArrayList<>()); + // verify write statuses + assertWriteStatuses(commitMetadata.getWriteStatuses(), 1, size / 2, fileAbsPaths, fileNames); + + // verify rows + Dataset result = sqlContext.read().parquet(fileAbsPaths.get().toArray(new String[0])); + assertOutput(inputRows, result, instantTime, fileNames); + } + + private void writeRows(Dataset inputRows, HoodieBulkInsertDataInternalWriter writer) + throws Exception { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java new file mode 100644 index 0000000000000..69829ec281a49 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/test/scala/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.spark3.internal; + +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.internal.HoodieBulkInsertInternalWriterTestBase; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.testutils.HoodieClientTestUtils; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.DataWriter; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; +import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows; + +/** + * Unit tests {@link HoodieDataSourceInternalBatchWrite}. + */ +public class TestHoodieDataSourceInternalBatchWrite extends + HoodieBulkInsertInternalWriterTestBase { + + @Test + public void testDataSourceWriter() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String instantTime = "001"; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong()); + + String[] partitionPaths = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS; + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(1000); + int batches = 5; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + List commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + + @Test + public void testMultipleDataSourceWrites() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + int partitionCounter = 0; + + // execute N rounds + for (int i = 0; i < 5; i++) { + String instantTime = "00" + i; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong()); + + int size = 10 + RANDOM.nextInt(1000); + int batches = 5; // one batch per partition + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); + + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + } + + @Test + public void testLargeWrites() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + int partitionCounter = 0; + + // execute N rounds + for (int i = 0; i < 3; i++) { + String instantTime = "00" + i; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + List commitMessages = new ArrayList<>(); + Dataset totalInputRows = null; + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(partitionCounter++, RANDOM.nextLong()); + + int size = 10000 + RANDOM.nextInt(10000); + int batches = 3; // one batch per partition + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages.add(commitMetadata); + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime); + + // verify output + assertOutput(totalInputRows, result, instantTime, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + } + } + + /** + * Tests that DataSourceWriter.abort() will abort the written records of interest write and commit batch1 write and abort batch2 Read of entire dataset should show only records from batch1. + * commit batch1 + * abort batch2 + * verify only records from batch1 is available to read + */ + @Test + public void testAbort() throws Exception { + // init config and table + HoodieWriteConfig cfg = getConfigBuilder(basePath).build(); + HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); + String instantTime0 = "00" + 0; + // init writer + HoodieDataSourceInternalBatchWrite dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime0, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + + DataWriter writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(0, RANDOM.nextLong()); + + List partitionPaths = Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS); + List partitionPathsAbs = new ArrayList<>(); + for (String partitionPath : partitionPaths) { + partitionPathsAbs.add(basePath + "/" + partitionPath + "/*"); + } + + int size = 10 + RANDOM.nextInt(100); + int batches = 1; + Dataset totalInputRows = null; + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + if (totalInputRows == null) { + totalInputRows = inputRows; + } else { + totalInputRows = totalInputRows.union(inputRows); + } + } + + HoodieWriterCommitMessage commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + List commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + // commit 1st batch + dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + Dataset result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify rows + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + assertWriteStatuses(commitMessages.get(0).getWriteStatuses(), batches, size, Option.empty(), Option.empty()); + + // 2nd batch. abort in the end + String instantTime1 = "00" + 1; + dataSourceInternalBatchWrite = + new HoodieDataSourceInternalBatchWrite(instantTime1, cfg, STRUCT_TYPE, sqlContext.sparkSession(), hadoopConf); + writer = dataSourceInternalBatchWrite.createBatchWriterFactory(null).createWriter(1, RANDOM.nextLong()); + + for (int j = 0; j < batches; j++) { + String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[j % 3]; + Dataset inputRows = getRandomRows(sqlContext, size, partitionPath, false); + writeRows(inputRows, writer); + } + + commitMetadata = (HoodieWriterCommitMessage) writer.commit(); + commitMessages = new ArrayList<>(); + commitMessages.add(commitMetadata); + // commit 1st batch + dataSourceInternalBatchWrite.abort(commitMessages.toArray(new HoodieWriterCommitMessage[0])); + metaClient.reloadActiveTimeline(); + result = HoodieClientTestUtils.read(jsc, basePath, sqlContext, metaClient.getFs(), partitionPathsAbs.toArray(new String[0])); + // verify rows + // only rows from first batch should be present + assertOutput(totalInputRows, result, instantTime0, Option.empty()); + } + + private void writeRows(Dataset inputRows, DataWriter writer) throws Exception { + List internalRows = toInternalRows(inputRows, ENCODER); + // issue writes + for (InternalRow internalRow : internalRows) { + writer.write(internalRow); + } + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index fa223980d663e..6c8fd8f916483 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -84,6 +84,9 @@ public class HiveSyncConfig implements Serializable { + "Disabled by default for backward compatibility.") public Boolean supportTimestamp = false; + @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") + public Boolean decodePartition = false; + public static HiveSyncConfig copy(HiveSyncConfig cfg) { HiveSyncConfig newConfig = new HiveSyncConfig(); newConfig.basePath = cfg.basePath; @@ -97,15 +100,30 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.tableName = cfg.tableName; newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; newConfig.supportTimestamp = cfg.supportTimestamp; + newConfig.decodePartition = cfg.decodePartition; return newConfig; } @Override public String toString() { - return "HiveSyncConfig{databaseName='" + databaseName + '\'' + ", tableName='" + tableName + '\'' - + ", hiveUser='" + hiveUser + '\'' + ", hivePass='" + hivePass + '\'' + ", jdbcUrl='" + jdbcUrl + '\'' - + ", basePath='" + basePath + '\'' + ", partitionFields=" + partitionFields + ", partitionValueExtractorClass='" - + partitionValueExtractorClass + '\'' + ", assumeDatePartitioning=" + assumeDatePartitioning + '\'' + ", supportTimestamp='" + supportTimestamp + '\'' - + ", usePreApacheInputFormat=" + usePreApacheInputFormat + ", useJdbc=" + useJdbc + ", help=" + help + '}'; + return "HiveSyncConfig{" + + "databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", baseFileFormat='" + baseFileFormat + '\'' + + ", hiveUser='" + hiveUser + '\'' + + ", hivePass='" + hivePass + '\'' + + ", jdbcUrl='" + jdbcUrl + '\'' + + ", basePath='" + basePath + '\'' + + ", partitionFields=" + partitionFields + + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' + + ", assumeDatePartitioning=" + assumeDatePartitioning + + ", usePreApacheInputFormat=" + usePreApacheInputFormat + + ", useJdbc=" + useJdbc + + ", autoCreateDatabase=" + autoCreateDatabase + + ", skipROSuffix=" + skipROSuffix + + ", help=" + help + + ", supportTimestamp=" + supportTimestamp + + ", decodePartition=" + decodePartition + + '}'; } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 6d85395d3f6e5..88f4c10611b08 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -18,6 +18,9 @@ package org.apache.hudi.hive; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; @@ -162,7 +165,17 @@ private String getPartitionClause(String partition) { + ". Check partition strategy. "); List partBuilder = new ArrayList<>(); for (int i = 0; i < syncConfig.partitionFields.size(); i++) { - partBuilder.add("`" + syncConfig.partitionFields.get(i) + "`='" + partitionValues.get(i) + "'"); + String partitionValue = partitionValues.get(i); + // decode the partition before sync to hive to prevent multiple escapes of HIVE + if (syncConfig.decodePartition) { + try { + // This is a decode operator for encode in KeyGenUtils#getRecordPartitionPath + partitionValue = URLDecoder.decode(partitionValue, StandardCharsets.UTF_8.toString()); + } catch (UnsupportedEncodingException e) { + throw new HoodieHiveSyncException("error in decode partition: " + partitionValue, e); + } + } + partBuilder.add("`" + syncConfig.partitionFields.get(i) + "`='" + partitionValue + "'"); } return String.join(",", partBuilder); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java index 43f2ff27d2eff..7542755b2491e 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java @@ -46,9 +46,9 @@ public static class Config { private final FileSystem fs; - private final Schema sourceSchema; + protected Schema sourceSchema; - private Schema targetSchema; + protected Schema targetSchema; public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) { super(props, jssc); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 9b0097e8ba257..6966e2cd66490 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -33,6 +33,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; @@ -622,23 +623,14 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir String tableBasePath = dfsBasePath + "/" + tempDir; // Keep it higher than batch-size to test continuous mode int totalRecords = 3000; - // Initial bulk insert HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); cfg.continuousMode = true; 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_PROP)); - HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); - Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { - try { - ds.sync(); - } catch (Exception ex) { - throw new RuntimeException(ex.getMessage(), ex); - } - }); - TestHelpers.waitTillCondition((r) -> { + deltaStreamerTestRunner(cfg, (r) -> { if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { TestHelpers.assertAtleastNDeltaCommits(5, tableBasePath, dfs); TestHelpers.assertAtleastNCompactionCommits(2, tableBasePath, dfs); @@ -648,11 +640,48 @@ private void testUpsertsContinuousMode(HoodieTableType tableType, String tempDir TestHelpers.assertRecordCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); TestHelpers.assertDistanceCount(totalRecords, tableBasePath + "/*/*.parquet", sqlContext); return true; - }, 180); + }); + } + + private void deltaStreamerTestRunner(HoodieDeltaStreamer.Config cfg, Function condition) throws Exception { + HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); + Future dsFuture = Executors.newSingleThreadExecutor().submit(() -> { + try { + ds.sync(); + } catch (Exception ex) { + throw new RuntimeException(ex.getMessage(), ex); + } + }); + + TestHelpers.waitTillCondition(condition, 180); ds.shutdownGracefully(); dsFuture.get(); } + @Test + public void testInlineClustering() throws Exception { + String tableBasePath = dfsBasePath + "/inlineClustering"; + // Keep it higher than batch-size to test continuous mode + int totalRecords = 3000; + + // Initial bulk insert + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.UPSERT); + cfg.continuousMode = true; + cfg.tableType = HoodieTableType.MERGE_ON_READ.name(); + cfg.configs.add(String.format("%s=%d", SourceConfigs.MAX_UNIQUE_RECORDS_PROP, totalRecords)); + cfg.configs.add(String.format("%s=false", HoodieCompactionConfig.AUTO_CLEAN_PROP)); + cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true")); + cfg.configs.add(String.format("%s=%s", HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP, "2")); + + deltaStreamerTestRunner(cfg, (r) -> { + HoodieTableMetaClient metaClient = new HoodieTableMetaClient(this.dfs.getConf(), tableBasePath, true); + int pendingReplaceSize = metaClient.getActiveTimeline().filterPendingReplaceTimeline().getInstants().toArray().length; + int completeReplaceSize = metaClient.getActiveTimeline().getCompletedReplaceTimeline().getInstants().toArray().length; + LOG.info("PendingReplaceSize=" + pendingReplaceSize + ",completeReplaceSize = " + completeReplaceSize); + return completeReplaceSize > 0; + }); + } + /** * Test Bulk Insert and upserts with hive syncing. Tests Hudi incremental processing using a 2 step pipeline The first * step involves using a SQL template to transform a source TEST-DATA-SOURCE ============================> HUDI TABLE diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 80e828bf48d72..8b158c2242699 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -105,7 +105,6 @@ io.prometheus:simpleclient_common com.yammer.metrics:metrics-core org.apache.flink:flink-connector-kafka_${scala.binary.version} - org.apache.flink:flink-connector-kafka-base_${scala.binary.version} org.apache.kafka:kafka_${scala.binary.version} com.101tec:zkclient org.apache.kafka:kafka-clients @@ -191,12 +190,6 @@ flink-connector-kafka_${scala.binary.version} compile - - org.apache.flink - flink-connector-kafka-base_${scala.binary.version} - ${flink.version} - compile - diff --git a/pom.xml b/pom.xml index 8f170883ffc93..bb6085516107d 100644 --- a/pom.xml +++ b/pom.xml @@ -87,6 +87,7 @@ 2.6.7.3 2.6.7.1 2.7.4 + 2.10.0 2.0.0 2.17 1.10.1 @@ -105,7 +106,7 @@ 0.8.0 4.4.1 2.4.4 - 1.11.2 + 1.12.0 2.4.4 3.0.0 1.8.2 @@ -1355,10 +1356,10 @@ ${scala12.version} 2.12 2.4.1 - 2.10.0 - 2.10.0 - 2.10.0 - 2.10.0 + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version} + ${fasterxml.spark3.version}