diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 2fdf4295d7790..1734cb53c1160 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -34,8 +34,6 @@ stages: jobs: - job: unit_tests_spark_client steps: - - script: | - mvn $(MAVEN_OPTS) clean install -DskipTests - task: Cache@2 inputs: key: 'maven | "$(Agent.OS)" | **/pom.xml' @@ -44,6 +42,8 @@ stages: maven path: $(MAVEN_CACHE_FOLDER) displayName: Cache Maven local repo + - script: | + mvn $(MAVEN_OPTS) clean install -DskipTests - task: Maven@3 inputs: mavenPomFile: 'pom.xml' @@ -58,8 +58,6 @@ stages: mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: unit_tests_utilities steps: - - script: | - mvn $(MAVEN_OPTS) clean install -DskipTests - task: Cache@2 inputs: key: 'maven | "$(Agent.OS)" | **/pom.xml' @@ -68,6 +66,8 @@ stages: maven path: $(MAVEN_CACHE_FOLDER) displayName: Cache Maven local repo + - script: | + mvn $(MAVEN_OPTS) clean install -DskipTests - task: Maven@3 inputs: mavenPomFile: 'pom.xml' @@ -82,8 +82,6 @@ stages: mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: unit_tests_other_modules steps: - - script: | - mvn $(MAVEN_OPTS) clean install -DskipTests - task: Cache@2 inputs: key: 'maven | "$(Agent.OS)" | **/pom.xml' @@ -92,6 +90,8 @@ stages: maven path: $(MAVEN_CACHE_FOLDER) displayName: Cache Maven local repo + - script: | + mvn $(MAVEN_OPTS) clean install -DskipTests - task: Maven@3 inputs: mavenPomFile: 'pom.xml' diff --git a/docker/demo/config/test-suite/templates/clustering.yaml.template b/docker/demo/config/test-suite/templates/clustering.yaml.template index 7b33423e73d5c..fab10ecf4cab9 100644 --- a/docker/demo/config/test-suite/templates/clustering.yaml.template +++ b/docker/demo/config/test-suite/templates/clustering.yaml.template @@ -13,6 +13,8 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# +# yaml to test clustering dag_name: NAME-clustering.yaml dag_rounds: clustering_num_iterations dag_intermittent_delay_mins: clustering_delay_in_mins diff --git a/docker/demo/config/test-suite/templates/long-running.yaml.template b/docker/demo/config/test-suite/templates/long_test_suite.yaml.template similarity index 90% rename from docker/demo/config/test-suite/templates/long-running.yaml.template rename to docker/demo/config/test-suite/templates/long_test_suite.yaml.template index b6392967e3d0e..0715eb27e10e8 100644 --- a/docker/demo/config/test-suite/templates/long-running.yaml.template +++ b/docker/demo/config/test-suite/templates/long_test_suite.yaml.template @@ -13,8 +13,11 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# +# Long running test suite which cleans up input after every round of a dag. Which means, validation +# happens only for 1 round of dag everytime (as input is cleaned up) dag_name: NAME-long-running-multi-partitions.yaml -dag_rounds: num_iterations +dag_rounds: long_num_iterations dag_intermittent_delay_mins: delay_in_mins dag_content: first_insert: @@ -82,7 +85,7 @@ dag_content: deps: second_hive_sync last_validate: config: - execute_itr_count: 50 + execute_itr_count: long_num_iterations validate_clean: true validate_archival: true type: ValidateAsyncOperations diff --git a/docker/demo/config/test-suite/templates/medium_test_suite.yaml.template b/docker/demo/config/test-suite/templates/medium_test_suite.yaml.template new file mode 100644 index 0000000000000..b499a92fa692d --- /dev/null +++ b/docker/demo/config/test-suite/templates/medium_test_suite.yaml.template @@ -0,0 +1,92 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Long running test suite which validates entire input after every dag. Input accumulates and so validation +# happens for entire dataset. +dag_name: NAME-long-running-multi-partitions.yaml +dag_rounds: medium_num_iterations +dag_intermittent_delay_mins: delay_in_mins +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 1000 + num_partitions_insert: 50 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 1000 + num_partitions_insert: 2 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: third_insert + first_validate: + config: + validate_hive: true + type: ValidateDatasetNode + deps: first_hive_sync + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 2 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 50 + num_records_delete: 8000 + type: DeleteNode + deps: first_upsert + second_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_delete + second_validate: + config: + validate_hive: true + delete_input_data: false + type: ValidateDatasetNode + deps: second_hive_sync + last_validate: + config: + execute_itr_count: medium_num_iterations + validate_clean: true + validate_archival: true + type: ValidateAsyncOperations + deps: second_validate diff --git a/docker/demo/config/test-suite/templates/sanity.yaml.template b/docker/demo/config/test-suite/templates/sanity.yaml.template new file mode 100644 index 0000000000000..eae83b6af38ad --- /dev/null +++ b/docker/demo/config/test-suite/templates/sanity.yaml.template @@ -0,0 +1,83 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Sanity yaml to test simple operations. +dag_name: NAME-sanity.yaml +dag_rounds: 1 +dag_intermittent_delay_mins: delay_in_mins +dag_content: + first_insert: + config: + record_size: 1000 + num_partitions_insert: 5 + repeat_count: 1 + num_records_insert: 1000 + type: InsertNode + deps: none + second_insert: + config: + record_size: 1000 + num_partitions_insert: 50 + repeat_count: 1 + num_records_insert: 10000 + deps: first_insert + type: InsertNode + third_insert: + config: + record_size: 1000 + num_partitions_insert: 2 + repeat_count: 1 + num_records_insert: 300 + deps: second_insert + type: InsertNode + first_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: third_insert + first_validate: + config: + validate_hive: true + type: ValidateDatasetNode + deps: first_hive_sync + first_upsert: + config: + record_size: 1000 + num_partitions_insert: 2 + num_records_insert: 300 + repeat_count: 1 + num_records_upsert: 100 + num_partitions_upsert: 1 + type: UpsertNode + deps: first_validate + first_delete: + config: + num_partitions_delete: 50 + num_records_delete: 8000 + type: DeleteNode + deps: first_upsert + second_hive_sync: + config: + queue_name: "adhoc" + engine: "mr" + type: HiveSyncNode + deps: first_delete + second_validate: + config: + validate_hive: true + type: ValidateDatasetNode + deps: second_hive_sync diff --git a/docker/generate_test_suite.sh b/docker/generate_test_suite.sh index 60655a7a5169d..d7c1405630f0a 100755 --- a/docker/generate_test_suite.sh +++ b/docker/generate_test_suite.sh @@ -16,13 +16,17 @@ # See the License for the specific language governing permissions and # limitations under the License. -NUM_ITR=1 +MEDIUM_NUM_ITR=20 +LONG_NUM_ITR=50 DELAY_MINS=1 TABLE_TYPE=COPY_ON_WRITE +INCLUDE_LONG_TEST_SUITE=false +INCLUDE_MEDIUM_TEST_SUITE=false INCLUDE_CLUSTER_YAML=false -CLUSTER_NUM_ITR=2 +CLUSTER_NUM_ITR=30 CLUSTER_DELAY_MINS=1 -CLUSTER_ITR_COUNT=1 +CLUSTER_ITR_COUNT=15 +EXECUTE_TEST_SUITE=true JAR_NAME=hudi-integ-test-bundle-0.9.0-SNAPSHOT.jar INPUT_PATH="/user/hive/warehouse/hudi-integ-test-suite/input/" OUTPUT_PATH="/user/hive/warehouse/hudi-integ-test-suite/output/" @@ -35,8 +39,18 @@ do key="$1" case $key in - --num_iterations) - NUM_ITR="$2" + --execute_test_suite) + EXECUTE_TEST_SUITE="$2" + shift # past argument + shift # past value + ;; + --medium_num_iterations) + MEDIUM_NUM_ITR="$2" + shift # past argument + shift # past value + ;; + --long_num_iterations) + LONG_NUM_ITR="$2" shift # past argument shift # past value ;; @@ -50,6 +64,16 @@ case $key in shift # past argument shift # past value ;; + --include_long_test_suite_yaml) + INCLUDE_LONG_TEST_SUITE="$2" + shift # past argument + shift # past value + ;; + --include_medium_test_suite_yaml) + INCLUDE_MEDIUM_TEST_SUITE="$2" + shift # past argument + shift # past value + ;; --include_cluster_yaml) INCLUDE_CLUSTER_YAML="$2" shift # past argument @@ -97,13 +121,23 @@ esac done set -- "${POSITIONAL[@]}" # restore positional parameters -echo "Num Iterations = ${NUM_ITR}" +echo "Include Medium test suite $INCLUDE_MEDIUM_TEST_SUITE" +if $INCLUDE_MEDIUM_TEST_SUITE ; then + echo "Medium test suite iterations = ${MEDIUM_NUM_ITR}" +fi +echo "Include Long test suite $INCLUDE_LONG_TEST_SUITE" +if $INCLUDE_LONG_TEST_SUITE ; then + echo "Long test suite iterations = ${LONG_NUM_ITR}" +fi echo "Intermittent delay in mins = ${DELAY_MINS}" echo "Table type = ${TABLE_TYPE}" + echo "Include cluster yaml $INCLUDE_CLUSTER_YAML" -echo "Cluster total itr count $CLUSTER_NUM_ITR" -echo "Cluster delay mins $CLUSTER_DELAY_MINS" -echo "Cluster exec itr count $CLUSTER_ITR_COUNT" +if $INCLUDE_CLUSTER_YAML ; then + echo "Cluster total itr count $CLUSTER_NUM_ITR" + echo "Cluster delay mins $CLUSTER_DELAY_MINS" + echo "Cluster exec itr count $CLUSTER_ITR_COUNT" +fi echo "Jar name $JAR_NAME" INPUT_PATH=$(echo "$INPUT_PATH" | sed "s|\/|\\\/|g") echo "Input path $INPUT_PATH" @@ -125,22 +159,56 @@ if [ ! -d "demo/config/test-suite/staging" ]; then mkdir demo/config/test-suite/staging fi -cp demo/config/test-suite/templates/long-running.yaml.template demo/config/test-suite/staging/long-running.yaml +cp demo/config/test-suite/templates/sanity.yaml.template demo/config/test-suite/staging/sanity.yaml -sed -i '' "s/NAME/$TABLE_TYPE/" demo/config/test-suite/staging/long-running.yaml -sed -i '' "s/num_iterations/$NUM_ITR/" demo/config/test-suite/staging/long-running.yaml -sed -i '' "s/delay_in_mins/$DELAY_MINS/" demo/config/test-suite/staging/long-running.yaml +sed -i '' "s/NAME/$TABLE_TYPE/" demo/config/test-suite/staging/sanity.yaml cp demo/config/test-suite/templates/test.properties.template demo/config/test-suite/staging/test.properties sed -i '' "s/INPUT_PATH/$INPUT_PATH/" demo/config/test-suite/staging/test.properties -cp demo/config/test-suite/templates/spark_command.txt.template demo/config/test-suite/staging/long_running_spark_command.sh +cp demo/config/test-suite/templates/spark_command.txt.template demo/config/test-suite/staging/sanity_spark_command.sh + +sed -i '' "s/JAR_NAME/$JAR_NAME/" demo/config/test-suite/staging/sanity_spark_command.sh +sed -i '' "s/INPUT_PATH/$INPUT_PATH/" demo/config/test-suite/staging/sanity_spark_command.sh +sed -i '' "s/OUTPUT_PATH/$OUTPUT_PATH/" demo/config/test-suite/staging/sanity_spark_command.sh +sed -i '' "s/input_yaml/sanity.yaml/" demo/config/test-suite/staging/sanity_spark_command.sh +sed -i '' "s/TABLE_TYPE/$TABLE_TYPE/" demo/config/test-suite/staging/sanity_spark_command.sh -sed -i '' "s/JAR_NAME/$JAR_NAME/" demo/config/test-suite/staging/long_running_spark_command.sh -sed -i '' "s/INPUT_PATH/$INPUT_PATH/" demo/config/test-suite/staging/long_running_spark_command.sh -sed -i '' "s/OUTPUT_PATH/$OUTPUT_PATH/" demo/config/test-suite/staging/long_running_spark_command.sh -sed -i '' "s/input_yaml/long-running.yaml/" demo/config/test-suite/staging/long_running_spark_command.sh -sed -i '' "s/TABLE_TYPE/$TABLE_TYPE/" demo/config/test-suite/staging/long_running_spark_command.sh +if $INCLUDE_MEDIUM_TEST_SUITE ; then + + cp demo/config/test-suite/templates/medium_test_suite.yaml.template demo/config/test-suite/staging/medium_test_suite.yaml + + sed -i '' "s/NAME/$TABLE_TYPE/" demo/config/test-suite/staging/medium_test_suite.yaml + sed -i '' "s/medium_num_iterations/$MEDIUM_NUM_ITR/" demo/config/test-suite/staging/medium_test_suite.yaml + sed -i '' "s/delay_in_mins/$DELAY_MINS/" demo/config/test-suite/staging/medium_test_suite.yaml + + cp demo/config/test-suite/templates/spark_command.txt.template demo/config/test-suite/staging/medium_test_suite_spark_command.sh + + sed -i '' "s/JAR_NAME/$JAR_NAME/" demo/config/test-suite/staging/medium_test_suite_spark_command.sh + sed -i '' "s/INPUT_PATH/$INPUT_PATH/" demo/config/test-suite/staging/medium_test_suite_spark_command.sh + sed -i '' "s/OUTPUT_PATH/$OUTPUT_PATH/" demo/config/test-suite/staging/medium_test_suite_spark_command.sh + sed -i '' "s/input_yaml/medium_test_suite.yaml/" demo/config/test-suite/staging/medium_test_suite_spark_command.sh + sed -i '' "s/TABLE_TYPE/$TABLE_TYPE/" demo/config/test-suite/staging/medium_test_suite_spark_command.sh + +fi + +if $INCLUDE_LONG_TEST_SUITE ; then + + cp demo/config/test-suite/templates/long_test_suite.yaml.template demo/config/test-suite/staging/long_test_suite.yaml + + sed -i '' "s/NAME/$TABLE_TYPE/" demo/config/test-suite/staging/long_test_suite.yaml + sed -i '' "s/long_num_iterations/$LONG_NUM_ITR/" demo/config/test-suite/staging/long_test_suite.yaml + sed -i '' "s/delay_in_mins/$DELAY_MINS/" demo/config/test-suite/staging/long_test_suite.yaml + + cp demo/config/test-suite/templates/spark_command.txt.template demo/config/test-suite/staging/long_test_suite_spark_command.sh + + sed -i '' "s/JAR_NAME/$JAR_NAME/" demo/config/test-suite/staging/long_test_suite_spark_command.sh + sed -i '' "s/INPUT_PATH/$INPUT_PATH/" demo/config/test-suite/staging/long_test_suite_spark_command.sh + sed -i '' "s/OUTPUT_PATH/$OUTPUT_PATH/" demo/config/test-suite/staging/long_test_suite_spark_command.sh + sed -i '' "s/input_yaml/long_test_suite.yaml/" demo/config/test-suite/staging/long_test_suite_spark_command.sh + sed -i '' "s/TABLE_TYPE/$TABLE_TYPE/" demo/config/test-suite/staging/long_test_suite_spark_command.sh + +fi if $INCLUDE_CLUSTER_YAML ; then @@ -148,7 +216,7 @@ if $INCLUDE_CLUSTER_YAML ; then sed -i '' "s/NAME/$TABLE_TYPE/" demo/config/test-suite/staging/clustering.yaml sed -i '' "s/clustering_num_iterations/$CLUSTER_NUM_ITR/" demo/config/test-suite/staging/clustering.yaml - sed -i '' "s/clustering_delay_in_mins/$CLUSTER_DELAY_MINS/" demo/config/test-suite/staging/clustering.yaml + sed -i '' "s/delay_in_mins/$CLUSTER_DELAY_MINS/" demo/config/test-suite/staging/clustering.yaml sed -i '' "s/clustering_itr_count/$CLUSTER_ITR_COUNT/" demo/config/test-suite/staging/clustering.yaml cp demo/config/test-suite/templates/spark_command.txt.template demo/config/test-suite/staging/clustering_spark_command.sh @@ -162,11 +230,27 @@ if $INCLUDE_CLUSTER_YAML ; then fi -docker cp $CUR_DIR/../packaging/hudi-integ-test-bundle/target/$JAR_NAME adhoc-2:/opt/ -docker exec -it adhoc-2 /bin/bash rm -rf /opt/staging* -docker cp demo/config/test-suite/staging/ adhoc-2:/opt/ -docker exec -it adhoc-2 /bin/bash /opt/staging/long_running_spark_command.sh +if $EXECUTE_TEST_SUITE ; then + + docker cp $CUR_DIR/../packaging/hudi-integ-test-bundle/target/$JAR_NAME adhoc-2:/opt/ + docker exec -it adhoc-2 /bin/bash rm -rf /opt/staging* + docker cp demo/config/test-suite/staging/ adhoc-2:/opt/ + docker exec -it adhoc-2 /bin/bash echo "\n============================== Executing sanity test suite ============================== " + docker exec -it adhoc-2 /bin/bash /opt/staging/sanity_spark_command.sh + + if [ -f demo/config/test-suite/staging/medium_test_suite_spark_command.sh ]; then + docker exec -it adhoc-2 /bin/bash echo "\n\n\n============================== Executing medium test suite ============================== " + docker exec -it adhoc-2 /bin/bash /opt/staging/medium_test_suite_spark_command.sh + fi + + if [ -f demo/config/test-suite/staging/long_test_suite_spark_command.sh ]; then + docker exec -it adhoc-2 /bin/bash echo "\n\n\n============================== Executing long test suite ============================== " + docker exec -it adhoc-2 /bin/bash /opt/staging/long_test_suite_spark_command.sh + fi + + if [ -f demo/config/test-suite/staging/clustering_spark_command.sh ]; then + docker exec -it adhoc-2 /bin/bash echo "\n\n\n============================== Executing clustering test suite ============================== " + docker exec -it adhoc-2 /bin/bash /opt/staging/clustering_spark_command.sh + fi -if [ -f demo/config/test-suite/staging/clustering_spark_command.sh ]; then - docker exec -it adhoc-2 /bin/bash /opt/staging/clustering_spark_command.sh fi 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 39c1bc0e5fec5..b0b8d55d52aa5 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 @@ -176,8 +176,6 @@ public boolean commitStats(String instantTime, List stats, Opti HoodieTable table = createTable(config, hadoopConf); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getWriteSchema(), commitActionType); - // Finalize write - finalizeWrite(table, instantTime, stats); HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime)), lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); 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 9e89e0ef7f334..0556adac649d5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -35,6 +35,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.keygen.constant.KeyGeneratorType; @@ -52,6 +53,7 @@ import java.io.InputStream; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Properties; @@ -153,6 +155,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private static final String MERGE_ALLOW_DUPLICATE_ON_INSERTS = "hoodie.merge.allow.duplicate.on.inserts"; private static final String DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS = "false"; + // Enable usage of RocksDb for External Spillable Map + public static final String DEFAULT_SPILLABLE_DISK_MAP_TYPE = ExternalSpillableMap.DiskMapType.BITCASK.name(); + public static final String SPILLABLE_DISK_MAP_TYPE = "hoodie.spillable.diskmap.type"; + public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = "hoodie.client.heartbeat.interval_in_ms"; public static final Integer DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS = 60 * 1000; @@ -410,6 +416,10 @@ public boolean allowDuplicateInserts() { return Boolean.parseBoolean(props.getProperty(MERGE_ALLOW_DUPLICATE_ON_INSERTS)); } + public ExternalSpillableMap.DiskMapType getSpillableDiskMapType() { + return ExternalSpillableMap.DiskMapType.valueOf(props.getProperty(SPILLABLE_DISK_MAP_TYPE).toUpperCase(Locale.ROOT)); + } + public EngineType getEngineType() { return engineType; } @@ -1355,6 +1365,11 @@ public Builder withMergeAllowDuplicateOnInserts(boolean routeInsertsToNewFiles) return this; } + public Builder withSpillableDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) { + props.setProperty(SPILLABLE_DISK_MAP_TYPE, diskMapType.name()); + return this; + } + public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) { props.setProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs)); return this; @@ -1433,6 +1448,8 @@ protected void setDefaults() { MERGE_DATA_VALIDATION_CHECK_ENABLED, DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED); setDefaultOnCondition(props, !props.containsKey(MERGE_ALLOW_DUPLICATE_ON_INSERTS), MERGE_ALLOW_DUPLICATE_ON_INSERTS, DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS); + setDefaultOnCondition(props, !props.containsKey(SPILLABLE_DISK_MAP_TYPE), + SPILLABLE_DISK_MAP_TYPE, DEFAULT_SPILLABLE_DISK_MAP_TYPE); setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP), CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS)); setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index cbd77c60d1154..fd4980616b41b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -199,7 +199,8 @@ protected void initializeIncomingRecordsMap() { long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps()); LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge); this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(), - new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema)); + new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema), + config.getSpillableDiskMapType()); } catch (IOException io) { throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io); } @@ -231,7 +232,7 @@ protected void init(String fileId, Iterator> newRecordsItr) { LOG.info("Number of entries in MemoryBasedMap => " + ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries() + "Total size in bytes of MemoryBasedMap => " - + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + "Number of entries in DiskBasedMap => " + + ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + "Number of entries in BitCaskDiskMap => " + ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + "Size of file spilled to disk => " + ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 38a044799dfda..034465d9bb266 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -400,7 +400,7 @@ private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { // (re) init the metadata for reading. initTableMetadata(); try { - List instantsToSync = metadata.findInstantsToSync(); + List instantsToSync = metadata.findInstantsToSyncForWriter(); if (instantsToSync.isEmpty()) { return; } @@ -411,7 +411,7 @@ private void syncFromInstants(HoodieTableMetaClient datasetMetaClient) { for (HoodieInstant instant : instantsToSync) { LOG.info("Syncing instant " + instant + " to metadata table"); - Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, metadata.getSyncedInstantTime()); + Option> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, getLatestSyncedInstantTime()); if (records.isPresent()) { commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 02c5b9e646ad0..1b02a3b92b353 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -23,6 +23,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.util.Option; import java.io.Serializable; @@ -40,4 +41,9 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { void update(HoodieRestoreMetadata restoreMetadata, String instantTime); void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); + + /** + * Return the timestamp of the latest instant synced to the metadata table. + */ + Option getLatestSyncedInstantTime(); } 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 aa930f7b069da..05e4481ecfbe7 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 @@ -58,6 +58,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.compact.FlinkCompactHelpers; import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade; +import org.apache.hudi.util.FlinkClientUtil; import com.codahale.metrics.Timer; import org.apache.hadoop.conf.Configuration; @@ -174,7 +175,7 @@ public List insert(List> records, String instantTim /** * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table. * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param instantTime Instant time of the commit * @return list of WriteStatus to inspect errors and counts */ @@ -194,7 +195,7 @@ public List insertOverwrite( /** * Removes all existing records of the Hoodie table and inserts the given HoodieRecords, into the table. * - * @param records HoodieRecords to insert + * @param records HoodieRecords to insert * @param instantTime Instant time of the commit * @return list of WriteStatus to inspect errors and counts */ @@ -235,7 +236,7 @@ public List delete(List keys, String instantTime) { HoodieTable>, List, List> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime); preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); - HoodieWriteMetadata> result = table.delete(context,instantTime, keys); + HoodieWriteMetadata> result = table.delete(context, instantTime, keys); return postWrite(result, instantTime, table); } @@ -391,11 +392,11 @@ public void cleanHandlesGracefully() { /** * Get or create a new write handle in order to reuse the file handles. * - * @param record The first record in the bucket - * @param config Write config - * @param instantTime The instant time - * @param table The table - * @param recordItr Record iterator + * @param record The first record in the bucket + * @param config Write config + * @param instantTime The instant time + * @param table The table + * @param recordItr Record iterator * @return Existing write handle or create a new one */ private HoodieWriteHandle getOrCreateWriteHandle( @@ -454,7 +455,8 @@ public String getLastPendingInstant(HoodieTableType tableType) { } public String getLastPendingInstant(String actionType) { - HoodieTimeline unCompletedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterInflightsAndRequested(); + HoodieTimeline unCompletedTimeline = FlinkClientUtil.createMetaClient(basePath) + .getCommitsTimeline().filterInflightsAndRequested(); return unCompletedTimeline.getInstants() .filter(x -> x.getAction().equals(actionType)) .map(HoodieInstant::getTimestamp) @@ -465,7 +467,8 @@ public String getLastPendingInstant(String actionType) { public String getLastCompletedInstant(HoodieTableType tableType) { final String commitType = CommitUtils.getCommitActionType(tableType); - HoodieTimeline completedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterCompletedInstants(); + HoodieTimeline completedTimeline = FlinkClientUtil.createMetaClient(basePath) + .getCommitsTimeline().filterCompletedInstants(); return completedTimeline.getInstants() .filter(x -> x.getAction().equals(commitType)) .map(HoodieInstant::getTimestamp) @@ -475,8 +478,7 @@ public String getLastCompletedInstant(HoodieTableType tableType) { } public void transitionRequestedToInflight(String commitType, String inFlightInstant) { - HoodieFlinkTable table = getHoodieTable(); - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieActiveTimeline activeTimeline = FlinkClientUtil.createMetaClient(basePath).getActiveTimeline(); HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant); activeTimeline.transitionRequestedToInflight(requested, Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/FlinkClientUtil.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/FlinkClientUtil.java index 4112e2b52371b..65daf782fbaa8 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/FlinkClientUtil.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/util/FlinkClientUtil.java @@ -18,6 +18,8 @@ package org.apache.hudi.util; +import org.apache.hudi.common.table.HoodieTableMetaClient; + import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.Path; @@ -29,6 +31,13 @@ */ public class FlinkClientUtil { + /** + * Creates the meta client. + */ + public static HoodieTableMetaClient createMetaClient(String basePath) { + return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(FlinkClientUtil.getHadoopConf()).build(); + } + /** * Parses the file name from path. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 7c12a9e001024..c014e8be57976 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -29,6 +29,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -132,6 +134,23 @@ protected void commit(List records, String partitionName, String i }); } + /** + * Return the timestamp of the latest instant synced. + * + * To sync a instant on dataset, we create a corresponding delta-commit on the metadata table. So return the latest + * delta-commit. + */ + @Override + public Option getLatestSyncedInstantTime() { + if (!enabled) { + return Option.empty(); + } + + HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); + return timeline.getDeltaCommitTimeline().filterCompletedInstants() + .lastInstant().map(HoodieInstant::getTimestamp); + } + /** * Tag each record with the location. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java index fd41a16802f9f..ecb18c6bc2828 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.apache.avro.generic.IndexedRecord; @@ -38,6 +39,8 @@ import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -97,4 +100,51 @@ protected Integer getResult() { } } } + + @Test + public void testInterruptExecutor() { + final List hoodieRecords = dataGen.generateInserts(instantTime, 100); + + HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); + when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); + BoundedInMemoryQueueConsumer, Integer> consumer = + new BoundedInMemoryQueueConsumer, Integer>() { + + @Override + protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + try { + while (true) { + Thread.sleep(1000); + } + } catch (InterruptedException ie) { + return; + } + } + + @Override + protected void finish() { + } + + @Override + protected Integer getResult() { + return 0; + } + }; + + SparkBoundedInMemoryExecutor>, Integer> executor = null; + try { + executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, hoodieRecords.iterator(), consumer, + getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA)); + SparkBoundedInMemoryExecutor>, Integer> finalExecutor = executor; + + Thread.currentThread().interrupt(); + + assertThrows(HoodieException.class, () -> finalExecutor.execute()); + assertTrue(Thread.interrupted()); + } finally { + if (executor != null) { + executor.shutdownNow(); + } + } + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java index f34f567b19948..dc9999e25edda 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieKey; @@ -189,7 +190,8 @@ public void testOnlyValidPartitionsAdded() throws Exception { .addCommit("002").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10, 10, 10) .addInflightCommit("003").withBaseFilesInPartition("p1", 10).withBaseFilesInPartition("p2", 10); - final HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false) + final HoodieWriteConfig writeConfig = + getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.NEVER, true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).withDirectoryFilterRegex(filterDirRegex).build()).build(); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { client.startCommitWithTime("005"); @@ -491,6 +493,8 @@ public void testSync(HoodieTableType tableType) throws Exception { // Various table operations without metadata table enabled String restoreToInstant; + String inflightActionTimestamp; + String beforeInflightActionTimestamp; try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) { // updates newCommitTime = HoodieActiveTimeline.createNewInstantTime(); @@ -523,6 +527,10 @@ public void testSync(HoodieTableType tableType) throws Exception { assertTrue(metadata(client).isInSync()); } + // Record a timestamp for creating an inflight instance for sync testing + inflightActionTimestamp = HoodieActiveTimeline.createNewInstantTime(); + beforeInflightActionTimestamp = newCommitTime; + // Deletes newCommitTime = HoodieActiveTimeline.createNewInstantTime(); records = dataGen.generateDeletes(newCommitTime, 5); @@ -554,9 +562,41 @@ public void testSync(HoodieTableType tableType) throws Exception { assertTrue(metadata(client).isInSync()); } + // If there is an incomplete operation, the Metadata Table is not updated beyond that operations but the + // in-memory merge should consider all the completed operations. + Path inflightCleanPath = new Path(metaClient.getMetaPath(), HoodieTimeline.makeInflightCleanerFileName(inflightActionTimestamp)); + fs.create(inflightCleanPath).close(); + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { // Restore cannot be done until the metadata table is in sync. See HUDI-1502 for details client.syncTableMetadata(); + + // Table should sync only before the inflightActionTimestamp + HoodieBackedTableMetadataWriter writer = + (HoodieBackedTableMetadataWriter)SparkHoodieBackedTableMetadataWriter.create(hadoopConf, client.getConfig(), context); + assertEquals(writer.getLatestSyncedInstantTime().get(), beforeInflightActionTimestamp); + + // Reader should sync to all the completed instants + HoodieTableMetadata metadata = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(), + client.getConfig().getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR); + assertEquals(metadata.getSyncedInstantTime().get(), newCommitTime); + + // Remove the inflight instance holding back table sync + fs.delete(inflightCleanPath, false); + client.syncTableMetadata(); + + writer = + (HoodieBackedTableMetadataWriter)SparkHoodieBackedTableMetadataWriter.create(hadoopConf, client.getConfig(), context); + assertEquals(writer.getLatestSyncedInstantTime().get(), newCommitTime); + + // Reader should sync to all the completed instants + metadata = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(), + client.getConfig().getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR); + assertEquals(metadata.getSyncedInstantTime().get(), newCommitTime); + } + + // Enable metadata table and ensure it is synced + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) { client.restoreToInstant(restoreToInstant); assertFalse(metadata(client).isInSync()); @@ -956,11 +996,16 @@ private HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileList } private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { + return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics); + } + + private HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) { return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withAutoCommit(autoCommit) .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024) .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1) + .withFailedWritesCleaningPolicy(policy) .withAutoClean(false).retainCommits(1).retainFileVersions(1).build()) .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build()) .withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table") @@ -969,8 +1014,7 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, bool .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(useFileListingMetadata) - .enableMetrics(enableMetrics) - .enableFallback(false).build()) + .enableMetrics(enableMetrics).build()) .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) .withExecutorMetrics(true).usePrefix("unit-test").build()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java index 6aafeca530f2c..abd3ac51a20c2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java @@ -21,12 +21,12 @@ import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ReflectionUtils; import java.io.Serializable; import java.util.List; import java.util.Map; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.ReflectionUtils; /** * Bootstrap Index Interface. @@ -161,6 +161,6 @@ public abstract void appendNextPartition(String partitionPath, public static BootstrapIndex getBootstrapIndex(HoodieTableMetaClient metaClient) { return ((BootstrapIndex)(ReflectionUtils.loadClass( - metaClient.getTableConfig().getBootstrapIndexClass(), metaClient))); + metaClient.getTableConfig().getBootstrapIndexClass(), new Class[]{HoodieTableMetaClient.class}, metaClient))); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 1eff82f0ce81f..3a6b4f3b7aaf6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -67,10 +67,6 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig { public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained"; public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3; - // Controls whether or not, upon failure to fetch from metadata table, should fallback to listing. - public static final String ENABLE_FALLBACK_PROP = METADATA_PREFIX + ".fallback.enable"; - public static final String DEFAULT_ENABLE_FALLBACK = "true"; - // Regex to filter out matching directories during bootstrap public static final String DIRECTORY_FILTER_REGEX = METADATA_PREFIX + ".dir.filter.regex"; public static final String DEFAULT_DIRECTORY_FILTER_REGEX = ""; @@ -101,10 +97,6 @@ public boolean useFileListingMetadata() { return Boolean.parseBoolean(props.getProperty(METADATA_ENABLE_PROP)); } - public boolean enableFallback() { - return Boolean.parseBoolean(props.getProperty(ENABLE_FALLBACK_PROP)); - } - public boolean validateFileListingMetadata() { return Boolean.parseBoolean(props.getProperty(METADATA_VALIDATE_PROP)); } @@ -143,11 +135,6 @@ public Builder enableMetrics(boolean enableMetrics) { return this; } - public Builder enableFallback(boolean fallback) { - props.setProperty(ENABLE_FALLBACK_PROP, String.valueOf(fallback)); - return this; - } - public Builder validate(boolean validate) { props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate)); return this; @@ -218,8 +205,6 @@ public HoodieMetadataConfig build() { String.valueOf(DEFAULT_FILE_LISTING_PARALLELISM)); setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP), HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING); - setDefaultOnCondition(props, !props.containsKey(ENABLE_FALLBACK_PROP), ENABLE_FALLBACK_PROP, - DEFAULT_ENABLE_FALLBACK); setDefaultOnCondition(props, !props.containsKey(DIRECTORY_FILTER_REGEX), DIRECTORY_FILTER_REGEX, DEFAULT_DIRECTORY_FILTER_REGEX); return config; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index cf66f16744a68..3285a00ca0333 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -98,8 +98,8 @@ public class HoodieTableMetaClient implements Serializable { private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, - ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName) { + ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, + String payloadClassName) { LOG.info("Loading HoodieTableMetaClient from " + basePath); this.consistencyGuardConfig = consistencyGuardConfig; this.hadoopConf = new SerializableConfiguration(conf); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index 238b0c6e93f8d..170be49741499 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -99,7 +99,7 @@ protected void performScan() { LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries()); LOG.info( "Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize()); - LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); + LOG.info("Number of entries in BitCaskDiskMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries()); LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index c7d61a732bb3a..d4bf2dd209f5d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -127,7 +127,7 @@ public HoodieTimeline filterPendingReplaceTimeline() { @Override public HoodieTimeline filterPendingCompactionTimeline() { return new HoodieDefaultTimeline( - instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details); + instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index 57d11d0b26302..2895f46db249f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -87,7 +87,7 @@ public static Object loadClass(String clazz, Class[] constructorArgTypes, Obj try { return getClass(clazz).getConstructor(constructorArgTypes).newInstance(constructorArgs); } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - throw new HoodieException("Unable to instantiate class ", e); + throw new HoodieException("Unable to instantiate class " + clazz, e); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java index b6eab3cfb5621..2dbc9123f399b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java @@ -22,7 +22,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.util.collection.DiskBasedMap.FileEntry; +import org.apache.hudi.common.util.collection.BitCaskDiskMap.FileEntry; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.avro.generic.GenericRecord; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java similarity index 95% rename from hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java index fe4666305d3f6..b525289c96a2c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskBasedMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java @@ -52,11 +52,13 @@ * This class provides a disk spillable only map implementation. All of the data is currenly written to one file, * without any rollover support. It uses the following : 1) An in-memory map that tracks the key-> latest ValueMetadata. * 2) Current position in the file NOTE : Only String.class type supported for Key + * + * Inspired by https://github.com/basho/bitcask */ -public final class DiskBasedMap implements Map, Iterable { +public final class BitCaskDiskMap implements DiskMap { public static final int BUFFER_SIZE = 128 * 1024; // 128 KB - private static final Logger LOG = LogManager.getLogger(DiskBasedMap.class); + private static final Logger LOG = LogManager.getLogger(BitCaskDiskMap.class); // Stores the key and corresponding value's latest metadata spilled to disk private final Map valueMetadataMap; // Write only file @@ -76,7 +78,7 @@ public final class DiskBasedMap private transient Thread shutdownThread = null; - public DiskBasedMap(String baseFilePath) throws IOException { + public BitCaskDiskMap(String baseFilePath) throws IOException { this.valueMetadataMap = new ConcurrentHashMap<>(); this.writeOnlyFile = new File(baseFilePath, UUID.randomUUID().toString()); this.filePath = writeOnlyFile.getPath(); @@ -136,7 +138,7 @@ private void flushToDisk() { try { writeOnlyFileHandle.flush(); } catch (IOException e) { - throw new HoodieIOException("Failed to flush to DiskBasedMap file", e); + throw new HoodieIOException("Failed to flush to BitCaskDiskMap file", e); } } @@ -151,6 +153,7 @@ public Iterator iterator() { /** * Number of bytes spilled to disk. */ + @Override public long sizeOfFileOnDiskInBytes() { return filePosition.get(); } @@ -203,7 +206,7 @@ private synchronized R put(T key, R value, boolean flush) { Integer valueSize = val.length; Long timestamp = System.currentTimeMillis(); this.valueMetadataMap.put(key, - new DiskBasedMap.ValueMetadata(this.filePath, valueSize, filePosition.get(), timestamp)); + new BitCaskDiskMap.ValueMetadata(this.filePath, valueSize, filePosition.get(), timestamp)); byte[] serializedKey = SerializationUtils.serialize(key); filePosition .set(SpillableMapUtils.spillToDisk(writeOnlyFileHandle, new FileEntry(SpillableMapUtils.generateChecksum(val), @@ -287,6 +290,7 @@ public Collection values() { throw new HoodieException("Unsupported Operation Exception"); } + @Override public Stream valueStream() { final BufferedRandomAccessFile file = getRandomAccessFile(); return valueMetadataMap.values().stream().sorted().sequential().map(valueMetaData -> (R) get(valueMetaData, file)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskMap.java new file mode 100644 index 0000000000000..a3dd03c7083f4 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/DiskMap.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.common.util.collection; + +import java.io.Serializable; +import java.util.Map; +import java.util.stream.Stream; + +/** + * This interface provides the map interface for storing records in disk after they + * spill over from memory. Used by {@link ExternalSpillableMap}. + * + * @param The generic type of the keys + * @param The generic type of the values + */ +public interface DiskMap extends Map, Iterable { + /** + * @returns a stream of the values stored in the disk. + */ + Stream valueStream(); + + /** + * Number of bytes spilled to disk. + */ + long sizeOfFileOnDiskInBytes(); + + /** + * Cleanup. + */ + void close(); + +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java index 003d525b66d5e..18b25d793a2db 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java @@ -61,8 +61,8 @@ public class ExternalSpillableMap inMemoryMap; - // Map to store key-valuemetadata important to find the values spilled to disk - private transient volatile DiskBasedMap diskBasedMap; + // Map to store key-values on disk or db after it spilled over the memory + private transient volatile DiskMap diskBasedMap; // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and // incorrect payload estimation private final Double sizingFactorForInMemoryMap = 0.8; @@ -70,6 +70,8 @@ public class ExternalSpillableMap keySizeEstimator; // Size Estimator for key types private final SizeEstimator valueSizeEstimator; + // Type of the disk map + private final DiskMapType diskMapType; // current space occupied by this map in-memory private Long currentInMemoryMapSize; // An estimate of the size of each payload written to this map @@ -80,22 +82,35 @@ public class ExternalSpillableMap keySizeEstimator, - SizeEstimator valueSizeEstimator) throws IOException { + SizeEstimator valueSizeEstimator) throws IOException { + this(maxInMemorySizeInBytes, baseFilePath, keySizeEstimator, + valueSizeEstimator, DiskMapType.BITCASK); + } + + public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath, SizeEstimator keySizeEstimator, + SizeEstimator valueSizeEstimator, DiskMapType diskMapType) throws IOException { this.inMemoryMap = new HashMap<>(); this.baseFilePath = baseFilePath; - this.diskBasedMap = new DiskBasedMap<>(baseFilePath); this.maxInMemorySizeInBytes = (long) Math.floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap); this.currentInMemoryMapSize = 0L; this.keySizeEstimator = keySizeEstimator; this.valueSizeEstimator = valueSizeEstimator; + this.diskMapType = diskMapType; } - private DiskBasedMap getDiskBasedMap() { + private DiskMap getDiskBasedMap() { if (null == diskBasedMap) { synchronized (this) { if (null == diskBasedMap) { try { - diskBasedMap = new DiskBasedMap<>(baseFilePath); + switch (diskMapType) { + case ROCKS_DB: + diskBasedMap = new RocksDbDiskMap<>(baseFilePath); + break; + case BITCASK: + default: + diskBasedMap = new BitCaskDiskMap<>(baseFilePath); + } } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -113,7 +128,7 @@ public Iterator iterator() { } /** - * Number of entries in DiskBasedMap. + * Number of entries in BitCaskDiskMap. */ public int getDiskBasedMapNumEntries() { return getDiskBasedMap().size(); @@ -160,6 +175,14 @@ public boolean containsValue(Object value) { return inMemoryMap.containsValue(value) || getDiskBasedMap().containsValue(value); } + public boolean inMemoryContainsKey(Object key) { + return inMemoryMap.containsKey(key); + } + + public boolean inDiskContainsKey(Object key) { + return getDiskBasedMap().containsKey(key); + } + @Override public R get(Object key) { if (inMemoryMap.containsKey(key)) { @@ -259,6 +282,16 @@ public Set> entrySet() { return entrySet; } + /** + * The type of map to use for storing the Key, values on disk after it spills + * from memory in the {@link ExternalSpillableMap}. + */ + public enum DiskMapType { + BITCASK, + ROCKS_DB, + UNKNOWN + } + /** * Iterator that wraps iterating over all the values for this map 1) inMemoryIterator - Iterates over all the data * in-memory map 2) diskLazyFileIterator - Iterates over all the data spilled to disk. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/LazyFileIterable.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/LazyFileIterable.java index 95b1ac2b37b36..024f555424d3e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/LazyFileIterable.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/LazyFileIterable.java @@ -36,11 +36,11 @@ public class LazyFileIterable implements Iterable { // Used to access the value written at a specific position in the file private final String filePath; // Stores the key and corresponding value's latest metadata spilled to disk - private final Map inMemoryMetadataOfSpilledData; + private final Map inMemoryMetadataOfSpilledData; private transient Thread shutdownThread = null; - public LazyFileIterable(String filePath, Map map) { + public LazyFileIterable(String filePath, Map map) { this.filePath = filePath; this.inMemoryMetadataOfSpilledData = map; } @@ -61,16 +61,16 @@ public class LazyFileIterator implements Iterator { private final String filePath; private BufferedRandomAccessFile readOnlyFileHandle; - private final Iterator> metadataIterator; + private final Iterator> metadataIterator; - public LazyFileIterator(String filePath, Map map) throws IOException { + public LazyFileIterator(String filePath, Map map) throws IOException { this.filePath = filePath; - this.readOnlyFileHandle = new BufferedRandomAccessFile(filePath, "r", DiskBasedMap.BUFFER_SIZE); + this.readOnlyFileHandle = new BufferedRandomAccessFile(filePath, "r", BitCaskDiskMap.BUFFER_SIZE); readOnlyFileHandle.seek(0); // sort the map in increasing order of offset of value so disk seek is only in one(forward) direction this.metadataIterator = map.entrySet().stream() - .sorted((Map.Entry o1, Map.Entry o2) -> o1 + .sorted((Map.Entry o1, Map.Entry o2) -> o1 .getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue())) .collect(Collectors.toList()).iterator(); this.addShutdownHook(); @@ -90,8 +90,8 @@ public R next() { if (!hasNext()) { throw new IllegalStateException("next() called on EOF'ed stream. File :" + filePath); } - Map.Entry entry = this.metadataIterator.next(); - return DiskBasedMap.get(entry.getValue(), readOnlyFileHandle); + Map.Entry entry = this.metadataIterator.next(); + return BitCaskDiskMap.get(entry.getValue(), readOnlyFileHandle); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java index 3c08460f2339a..80533111b691f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDBDAO.java @@ -45,10 +45,12 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -64,11 +66,13 @@ public class RocksDBDAO { private transient RocksDB rocksDB; private boolean closed = false; private final String rocksDBBasePath; + private long totalBytesWritten; public RocksDBDAO(String basePath, String rocksDBBasePath) { this.rocksDBBasePath = String.format("%s/%s/%s", rocksDBBasePath, basePath.replace("/", "_"), UUID.randomUUID().toString()); init(); + totalBytesWritten = 0L; } /** @@ -169,7 +173,7 @@ public void writeBatch(BatchHandler handler) { */ public void putInBatch(WriteBatch batch, String columnFamilyName, String key, T value) { try { - byte[] payload = SerializationUtils.serialize(value); + byte[] payload = serializePayload(value); batch.put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload); } catch (Exception e) { throw new HoodieException(e); @@ -189,7 +193,7 @@ public void putInBatch(WriteBat K key, T value) { try { byte[] keyBytes = SerializationUtils.serialize(key); - byte[] payload = SerializationUtils.serialize(value); + byte[] payload = serializePayload(value); batch.put(managedHandlesMap.get(columnFamilyName), keyBytes, payload); } catch (Exception e) { throw new HoodieException(e); @@ -206,7 +210,7 @@ public void putInBatch(WriteBat */ public void put(String columnFamilyName, String key, T value) { try { - byte[] payload = SerializationUtils.serialize(value); + byte[] payload = serializePayload(value); getRocksDB().put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload); } catch (Exception e) { throw new HoodieException(e); @@ -223,7 +227,7 @@ public void put(String columnFamilyName, String key, T */ public void put(String columnFamilyName, K key, T value) { try { - byte[] payload = SerializationUtils.serialize(value); + byte[] payload = serializePayload(value); getRocksDB().put(managedHandlesMap.get(columnFamilyName), SerializationUtils.serialize(key), payload); } catch (Exception e) { throw new HoodieException(e); @@ -351,6 +355,10 @@ public Stream> prefixSearch(String colu return results.stream(); } + public Iterator iterator(String columnFamilyName) { + return new IteratorWrapper<>(getRocksDB().newIterator(managedHandlesMap.get(columnFamilyName))); + } + /** * Perform a prefix delete and return stream of key-value pairs retrieved. * @@ -448,10 +456,53 @@ public synchronized void close() { } } + public long getTotalBytesWritten() { + return totalBytesWritten; + } + + private byte[] serializePayload(T value) throws IOException { + byte[] payload = SerializationUtils.serialize(value); + totalBytesWritten += payload.length; + return payload; + } + String getRocksDBBasePath() { return rocksDBBasePath; } + /** + * {@link Iterator} wrapper for RocksDb Iterator {@link RocksIterator}. + */ + private static class IteratorWrapper implements Iterator { + + private final RocksIterator iterator; + + public IteratorWrapper(final RocksIterator iterator) { + this.iterator = iterator; + iterator.seekToFirst(); + } + + @Override + public boolean hasNext() { + return iterator.isValid(); + } + + @Override + public R next() { + if (!hasNext()) { + throw new IllegalStateException("next() called on rocksDB with no more valid entries"); + } + R val = SerializationUtils.deserialize(iterator.value()); + iterator.next(); + return val; + } + + @Override + public void forEachRemaining(Consumer action) { + action.accept(next()); + } + } + /** * Functional interface for stacking operation to Write batch. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDbDiskMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDbDiskMap.java new file mode 100644 index 0000000000000..734240b01ad6a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/RocksDbDiskMap.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.collection; + +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieNotSupportedException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.stream.Stream; + +/** + * This class provides a disk spillable only map implementation. + * All of the data is stored using the RocksDB implementation. + */ +public final class RocksDbDiskMap implements DiskMap { + // ColumnFamily allows partitioning data within RockDB, which allows + // independent configuration and faster deletes across partitions + // https://github.com/facebook/rocksdb/wiki/Column-Families + // For this use case, we use a single static column family/ partition + // + private static final String COLUMN_FAMILY_NAME = "spill_map"; + + private static final Logger LOG = LogManager.getLogger(RocksDbDiskMap.class); + // Stores the key and corresponding value's latest metadata spilled to disk + private final Set keySet; + private final String rocksDbStoragePath; + private RocksDBDAO rocksDb; + + public RocksDbDiskMap(String rocksDbStoragePath) throws IOException { + this.keySet = new HashSet<>(); + this.rocksDbStoragePath = rocksDbStoragePath; + } + + @Override + public int size() { + return keySet.size(); + } + + @Override + public boolean isEmpty() { + return keySet.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return keySet.contains((T) key); + } + + @Override + public boolean containsValue(Object value) { + throw new HoodieNotSupportedException("unable to compare values in map"); + } + + @Override + public R get(Object key) { + if (!containsKey(key)) { + return null; + } + return getRocksDb().get(COLUMN_FAMILY_NAME, (T) key); + } + + @Override + public R put(T key, R value) { + getRocksDb().put(COLUMN_FAMILY_NAME, key, value); + keySet.add(key); + return value; + } + + @Override + public R remove(Object key) { + R value = get(key); + if (value != null) { + keySet.remove((T) key); + getRocksDb().delete(COLUMN_FAMILY_NAME, (T) key); + } + return value; + } + + @Override + public void putAll(Map keyValues) { + getRocksDb().writeBatch(batch -> keyValues.forEach((key, value) -> getRocksDb().putInBatch(batch, COLUMN_FAMILY_NAME, key, value))); + keySet.addAll(keyValues.keySet()); + } + + @Override + public void clear() { + close(); + } + + @Override + public Set keySet() { + return keySet; + } + + @Override + public Collection values() { + throw new HoodieException("Unsupported Operation Exception"); + } + + @Override + public Set> entrySet() { + Set> entrySet = new HashSet<>(); + for (T key : keySet) { + entrySet.add(new AbstractMap.SimpleEntry<>(key, get(key))); + } + return entrySet; + } + + /** + * Custom iterator to iterate over values written to disk. + */ + @Override + public Iterator iterator() { + return getRocksDb().iterator(COLUMN_FAMILY_NAME); + } + + @Override + public Stream valueStream() { + return keySet.stream().map(key -> (R) get(key)); + } + + @Override + public long sizeOfFileOnDiskInBytes() { + return getRocksDb().getTotalBytesWritten(); + } + + @Override + public void close() { + keySet.clear(); + if (null != rocksDb) { + rocksDb.close(); + } + rocksDb = null; + } + + private RocksDBDAO getRocksDb() { + if (null == rocksDb) { + synchronized (this) { + if (null == rocksDb) { + rocksDb = new RocksDBDAO(COLUMN_FAMILY_NAME, rocksDbStoragePath); + rocksDb.addColumnFamily(COLUMN_FAMILY_NAME); + } + } + } + return rocksDb; + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index cec9ab61a9c1b..95a501f6e2a17 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -139,6 +139,10 @@ public E execute() { Future future = startConsumer(); // Wait for consumer to be done return future.get(); + } catch (InterruptedException ie) { + shutdownNow(); + Thread.currentThread().interrupt(); + throw new HoodieException(ie); } catch (Exception e) { throw new HoodieException(e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index b4143f336039a..85a4d69b7c55d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -62,6 +62,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { protected final HoodieMetadataConfig metadataConfig; // Directory used for Spillable Map when merging records protected final String spillableMapDirectory; + private String syncedInstantTime; protected boolean enabled; private TimelineMergedTableMetadata timelineMergedMetadata; @@ -101,11 +102,7 @@ public List getAllPartitionPaths() throws IOException { try { return fetchAllPartitionPaths(); } catch (Exception e) { - if (metadataConfig.enableFallback()) { - LOG.error("Failed to retrieve list of partition from metadata", e); - } else { - throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); - } + throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); } } return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, @@ -129,11 +126,7 @@ public FileStatus[] getAllFilesInPartition(Path partitionPath) try { return fetchAllFilesInPartition(partitionPath); } catch (Exception e) { - if (metadataConfig.enableFallback()) { - LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e); - } else { - throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e); - } + throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e); } } @@ -285,16 +278,44 @@ private Option> getMergedRecordByKey(String private void openTimelineScanner() { if (timelineMergedMetadata == null) { - List unSyncedInstants = findInstantsToSync(); + List unSyncedInstants = findInstantsToSyncForReader(); timelineMergedMetadata = new TimelineMergedTableMetadata(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), null); + + syncedInstantTime = unSyncedInstants.isEmpty() ? getLatestDatasetInstantTime() + : unSyncedInstants.get(unSyncedInstants.size() - 1).getTimestamp(); + } + } + + /** + * Return the timestamp of the latest synced instant. + */ + @Override + public Option getSyncedInstantTime() { + if (!enabled) { + return Option.empty(); } + + return Option.ofNullable(syncedInstantTime); } - protected abstract List findInstantsToSync(); + /** + * Return the instants which are not-synced to the {@code HoodieTableMetadata}. + * + * This is the list of all completed but un-synched instants. + */ + protected abstract List findInstantsToSyncForReader(); + + /** + * Return the instants which are not-synced to the {@code HoodieTableMetadataWriter}. + * + * This is the list of all completed but un-synched instants which do not have any incomplete instants in between them. + */ + protected abstract List findInstantsToSyncForWriter(); + @Override public boolean isInSync() { - return enabled && findInstantsToSync().isEmpty(); + return enabled && findInstantsToSyncForWriter().isEmpty(); } protected HoodieEngineContext getEngineContext() { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index 66e8f419e168e..f374d61b53031 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -29,7 +29,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -265,7 +264,22 @@ public synchronized void close() throws Exception { * Return an ordered list of instants which have not been synced to the Metadata Table. */ @Override - protected List findInstantsToSync() { + protected List findInstantsToSyncForReader() { + return findInstantsToSync(true); + } + + /** + * Return an ordered list of instants which have not been synced to the Metadata Table. + */ + @Override + protected List findInstantsToSyncForWriter() { + return findInstantsToSync(false); + } + + /** + * Return an ordered list of instants which have not been synced to the Metadata Table. + */ + private List findInstantsToSync(boolean ignoreIncompleteInstants) { initIfNeeded(); // if there are no instants yet, return empty list, since there is nothing to sync here. @@ -277,7 +291,8 @@ protected List findInstantsToSync() { // are candidates for sync. String latestMetadataInstantTime = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp(); HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE); - Option earliestIncompleteInstant = candidateTimeline.filterInflightsAndRequested().firstInstant(); + Option earliestIncompleteInstant = ignoreIncompleteInstants ? Option.empty() + : candidateTimeline.filterInflightsAndRequested().firstInstant(); if (earliestIncompleteInstant.isPresent()) { return candidateTimeline.filterCompletedInstants() @@ -289,20 +304,6 @@ protected List findInstantsToSync() { } } - /** - * Return the timestamp of the latest compaction instant. - */ - @Override - public Option getSyncedInstantTime() { - if (!enabled) { - return Option.empty(); - } - - HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline(); - return timeline.getDeltaCommitTimeline().filterCompletedInstants() - .lastInstant().map(HoodieInstant::getTimestamp); - } - public boolean enabled() { return enabled; } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java similarity index 96% rename from hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java rename to hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java index e3cc886568ffb..45aaff3770075 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestDiskBasedMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestBitCaskDiskMap.java @@ -57,9 +57,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue; /** - * Tests dis based map {@link DiskBasedMap}. + * Tests dis based map {@link BitCaskDiskMap}. */ -public class TestDiskBasedMap extends HoodieCommonTestHarness { +public class TestBitCaskDiskMap extends HoodieCommonTestHarness { @BeforeEach public void setup() { @@ -68,7 +68,7 @@ public void setup() { @Test public void testSimpleInsert() throws IOException, URISyntaxException { - DiskBasedMap records = new DiskBasedMap<>(basePath); + BitCaskDiskMap records = new BitCaskDiskMap<>(basePath); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -86,7 +86,7 @@ public void testSimpleInsert() throws IOException, URISyntaxException { @Test public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException { - DiskBasedMap records = new DiskBasedMap<>(basePath); + BitCaskDiskMap records = new BitCaskDiskMap<>(basePath); List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); Set recordKeys = new HashSet<>(); // insert generated records into the map @@ -109,7 +109,7 @@ public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISynta public void testSimpleUpsert() throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); - DiskBasedMap records = new DiskBasedMap<>(basePath); + BitCaskDiskMap records = new BitCaskDiskMap<>(basePath); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // perform some inserts @@ -189,7 +189,7 @@ public void testSizeEstimator() throws IOException, URISyntaxException { @Test public void testPutAll() throws IOException, URISyntaxException { - DiskBasedMap records = new DiskBasedMap<>(basePath); + BitCaskDiskMap records = new BitCaskDiskMap<>(basePath); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); Map recordMap = new HashMap<>(); iRecords.forEach(r -> { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java index b240c8dd14aa0..3113747ba8fe4 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestExternalSpillableMap.java @@ -38,6 +38,8 @@ import org.junit.jupiter.api.MethodOrderer.Alphanumeric; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestMethodOrder; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.io.IOException; import java.io.UncheckedIOException; @@ -65,12 +67,14 @@ public void setUp() { failureOutputPath = basePath + "/test_fail"; } - @Test - public void simpleInsertTest() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void simpleInsertTest(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -84,13 +88,15 @@ public void simpleInsertTest() throws IOException, URISyntaxException { } } - @Test - public void testSimpleUpsert() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void testSimpleUpsert(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -120,14 +126,16 @@ public void testSimpleUpsert() throws IOException, URISyntaxException { }); } - @Test - public void testAllMapOperations() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void testAllMapOperations(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); String payloadClazz = HoodieAvroPayload.class.getName(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); // insert a bunch of records so that values spill to disk too @@ -176,12 +184,14 @@ public void testAllMapOperations() throws IOException, URISyntaxException { assertTrue(records.size() == 0); } - @Test - public void simpleTestWithException() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void simpleTestWithException(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); ExternalSpillableMap> records = new ExternalSpillableMap<>(16L, - failureOutputPath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + failureOutputPath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); @@ -194,13 +204,15 @@ public void simpleTestWithException() throws IOException, URISyntaxException { }); } - @Test - public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk @@ -245,13 +257,15 @@ record = records.get(key); assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); } - @Test - public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException { + @ParameterizedTest + @EnumSource(ExternalSpillableMap.DiskMapType.class) + public void testDataCorrectnessWithoutHoodieMetadata(ExternalSpillableMap.DiskMapType diskMapType) throws IOException, URISyntaxException { Schema schema = SchemaTestUtil.getSimpleSchema(); ExternalSpillableMap> records = - new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema)); // 16B + new ExternalSpillableMap<>(16L, basePath, new DefaultSizeEstimator(), + new HoodieRecordSizeEstimator(schema), diskMapType); // 16B List recordKeys = new ArrayList<>(); // Ensure we spill to disk diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbBasedMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbBasedMap.java index 1111d10e43f37..5b71b5ec24235 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbBasedMap.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbBasedMap.java @@ -38,7 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; /** - * Tests RocksDB based map {@link RocksDBBasedMap}. + * Tests RocksDB based map {@link RocksDbDiskMap}. */ public class TestRocksDbBasedMap extends HoodieCommonTestHarness { @@ -49,7 +49,7 @@ public void setUp() { @Test public void testSimple() throws IOException, URISyntaxException { - RocksDBBasedMap records = new RocksDBBasedMap(basePath); + RocksDbDiskMap records = new RocksDbDiskMap(basePath); List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); ((GenericRecord) iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java new file mode 100644 index 0000000000000..5262f3c6d4307 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestRocksDbDiskMap.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.collection; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.SchemaTestUtil; +import org.apache.hudi.common.testutils.SpillableMapTestUtils; +import org.apache.hudi.common.util.Option; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; + +import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test the rocksDb based Map {@link RocksDbDiskMap} + * that is used by {@link ExternalSpillableMap}. + */ +public class TestRocksDbDiskMap extends HoodieCommonTestHarness { + + @BeforeEach + public void setUp() { + initPath(); + } + + @Test + public void testSimpleInsertSequential() throws IOException, URISyntaxException { + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List recordKeys = setupMapWithRecords(rocksDBBasedMap, 100); + + Iterator> itr = rocksDBBasedMap.iterator(); + List oRecords = new ArrayList<>(); + while (itr.hasNext()) { + HoodieRecord rec = itr.next(); + oRecords.add(rec); + assert recordKeys.contains(rec.getRecordKey()); + } + assertEquals(recordKeys.size(), oRecords.size()); + } + + @Test + public void testSimpleInsertRandomAccess() throws IOException, URISyntaxException { + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List recordKeys = setupMapWithRecords(rocksDBBasedMap, 100); + + Random random = new Random(); + for (int i = 0; i < recordKeys.size(); i++) { + String key = recordKeys.get(random.nextInt(recordKeys.size())); + assert rocksDBBasedMap.get(key) != null; + } + } + + @Test + public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException { + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000); + Set recordKeys = new HashSet<>(); + // insert generated records into the map + hoodieRecords.forEach(r -> { + rocksDBBasedMap.put(r.getRecordKey(), r); + recordKeys.add(r.getRecordKey()); + }); + // make sure records have spilled to disk + assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0); + Iterator> itr = rocksDBBasedMap.iterator(); + List oRecords = new ArrayList<>(); + while (itr.hasNext()) { + HoodieRecord rec = itr.next(); + oRecords.add(rec); + assert recordKeys.contains(rec.getRecordKey()); + } + } + + @Test + public void testSimpleUpsert() throws IOException, URISyntaxException { + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List insertedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + List recordKeys = SpillableMapTestUtils.upsertRecords(insertedRecords, rocksDBBasedMap); + String oldCommitTime = + ((GenericRecord) insertedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + + // generate updates from inserts for first 50 keys / subset of keys + List updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys.subList(0, 50), + SchemaTestUtil.generateHoodieTestRecords(0, 50), HoodieActiveTimeline.createNewInstantTime()); + String newCommitTime = + ((GenericRecord) updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + + // perform upserts + List updatedRecordKeys = SpillableMapTestUtils.upsertRecords(updatedRecords, rocksDBBasedMap); + + // Upserted records (on disk) should have the latest commit time + Iterator> itr = rocksDBBasedMap.iterator(); + while (itr.hasNext()) { + HoodieRecord rec = itr.next(); + try { + IndexedRecord indexedRecord = (IndexedRecord) rec.getData().getInsertValue(schema).get(); + String latestCommitTime = + ((GenericRecord) indexedRecord).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(); + assert recordKeys.contains(rec.getRecordKey()) || updatedRecordKeys.contains(rec.getRecordKey()); + if (updatedRecordKeys.contains(rec.getRecordKey())) { + assertEquals(latestCommitTime, newCommitTime); + } else { + assertEquals(latestCommitTime, oldCommitTime); + } + } catch (IOException io) { + throw new UncheckedIOException(io); + } + } + } + + @Test + public void testPutAll() throws IOException, URISyntaxException { + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100); + Map recordMap = new HashMap<>(); + iRecords.forEach(r -> { + String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString(); + HoodieRecord value = new HoodieRecord<>(new HoodieKey(key, partitionPath), new HoodieAvroPayload(Option.of((GenericRecord) r))); + recordMap.put(key, value); + }); + + rocksDBBasedMap.putAll(recordMap); + // make sure records have spilled to disk + assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0); + + // make sure all added records are present + for (Map.Entry entry : rocksDBBasedMap.entrySet()) { + assertTrue(recordMap.containsKey(entry.getKey())); + } + } + + @Test + public void testSimpleRemove() throws IOException, URISyntaxException { + RocksDbDiskMap rocksDBBasedMap = new RocksDbDiskMap<>(basePath); + List recordKeys = setupMapWithRecords(rocksDBBasedMap, 100); + + List deleteKeys = recordKeys.subList(0, 10); + for (String deleteKey : deleteKeys) { + assert rocksDBBasedMap.remove(deleteKey) != null; + assert rocksDBBasedMap.get(deleteKey) == null; + } + } + + private List setupMapWithRecords(RocksDbDiskMap rocksDBBasedMap, int numRecords) throws IOException, URISyntaxException { + List iRecords = SchemaTestUtil.generateHoodieTestRecords(0, numRecords); + List recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, rocksDBBasedMap); + // Ensure the number of records is correct + assertEquals(rocksDBBasedMap.size(), recordKeys.size()); + // make sure records have spilled to disk + assertTrue(rocksDBBasedMap.sizeOfFileOnDiskInBytes() > 0); + return recordKeys; + } +} \ No newline at end of file diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index feef0f3cc9a4e..bf1fc0b145648 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -142,12 +142,6 @@ private FlinkOptions() { + "2) payload_combine: read the base file records first, for each record in base file, checks whether the key is in the\n" + " log file records(combines the two records with same key for base and log file records), then read the left log file records"); - public static final ConfigOption HIVE_STYLE_PARTITION = ConfigOptions - .key("hoodie.datasource.hive_style_partition") - .booleanType() - .defaultValue(false) - .withDescription("Whether the partition path is with Hive style, e.g. '{partition key}={partition value}', default false"); - public static final ConfigOption UTC_TIMEZONE = ConfigOptions .key("read.utc-timezone") .booleanType() @@ -260,12 +254,20 @@ private FlinkOptions() { .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n" + "Actual value obtained by invoking .toString(), default ''"); - public static final ConfigOption PARTITION_PATH_URL_ENCODE = ConfigOptions - .key("write.partition.url_encode") + public static final ConfigOption URL_ENCODE_PARTITIONING = ConfigOptions + .key(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY) .booleanType() .defaultValue(false) .withDescription("Whether to encode the partition path url, default false"); + public static final ConfigOption HIVE_STYLE_PARTITIONING = ConfigOptions + .key(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY) + .booleanType() + .defaultValue(false) + .withDescription("Whether to use Hive style partitioning.\n" + + "If set true, the names of partition folders follow = format.\n" + + "By default false (the names of partition folders are only partition values)"); + public static final ConfigOption KEYGEN_CLASS = ConfigOptions .key(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP) .stringType() diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index 965e5737e8e9c..bffd7d2a251ab 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -32,11 +32,14 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.sink.event.BatchWriteSuccessEvent; +import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.table.action.commit.FlinkWriteHelper; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -77,20 +80,18 @@ * starts a new instant on the time line when a checkpoint triggers, the coordinator checkpoints always * start before its operator, so when this function starts a checkpoint, a REQUESTED instant already exists. * - *

In order to improve the throughput, The function process thread does not block data buffering - * after the checkpoint thread starts flushing the existing data buffer. So there is possibility that the next checkpoint - * batch was written to current checkpoint. When a checkpoint failure triggers the write rollback, there may be some duplicate records - * (e.g. the eager write batch), the semantics is still correct using the UPSERT operation. + *

The function process thread blocks data buffering after the checkpoint thread finishes flushing the existing data buffer until + * the current checkpoint succeed and the coordinator starts a new instant. Any error triggers the job failure during the metadata committing, + * when the job recovers from a failure, the write function re-send the write metadata to the coordinator to see if these metadata + * can re-commit, thus if unexpected error happens during the instant committing, the coordinator would retry to commit when the job + * recovers. * *

Fault Tolerance

* - *

The operator coordinator checks and commits the last instant then starts a new one when a checkpoint finished successfully. - * The operator rolls back the written data and throws to trigger a failover when any error occurs. - * This means one Hoodie instant may span one or more checkpoints(some checkpoints notifications may be skipped). - * If a checkpoint timed out, the next checkpoint would help to rewrite the left buffer data (clean the buffer in the last - * step of the #flushBuffer method). - * - *

The operator coordinator would try several times when committing the write status. + *

The operator coordinator checks and commits the last instant then starts a new one after a checkpoint finished successfully. + * It rolls back any inflight instant before it starts a new instant, this means one hoodie instant only span one checkpoint, + * the write function blocks data buffer flushing for the configured checkpoint timeout + * before it throws exception, any checkpoint failure would finally trigger the job failure. * *

Note: The function task requires the input stream be shuffled by the file IDs. * @@ -163,6 +164,16 @@ public class StreamWriteFunction */ private volatile boolean confirming = false; + /** + * List state of the write metadata events. + */ + private transient ListState writeMetadataState; + + /** + * Write status list for the current checkpoint. + */ + private List writeStatuses; + /** * Constructs a StreamingSinkFunction. * @@ -174,27 +185,43 @@ public StreamWriteFunction(Configuration config) { @Override public void open(Configuration parameters) throws IOException { - this.taskID = getRuntimeContext().getIndexOfThisSubtask(); - this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext()); - this.actionType = CommitUtils.getCommitActionType( - WriteOperationType.fromValue(config.getString(FlinkOptions.OPERATION)), - HoodieTableType.valueOf(config.getString(FlinkOptions.TABLE_TYPE))); this.tracer = new TotalSizeTracer(this.config); initBuffer(); initWriteFunction(); } @Override - public void initializeState(FunctionInitializationContext context) { - // no operation + public void initializeState(FunctionInitializationContext context) throws Exception { + this.taskID = getRuntimeContext().getIndexOfThisSubtask(); + this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext()); + this.actionType = CommitUtils.getCommitActionType( + WriteOperationType.fromValue(config.getString(FlinkOptions.OPERATION)), + HoodieTableType.valueOf(config.getString(FlinkOptions.TABLE_TYPE))); + + this.writeStatuses = new ArrayList<>(); + this.writeMetadataState = context.getOperatorStateStore().getListState( + new ListStateDescriptor<>( + "write-metadata-state", + TypeInformation.of(WriteMetadataEvent.class) + )); + + if (context.isRestored()) { + restoreWriteMetadata(); + } else { + sendBootstrapEvent(); + } + // blocks flushing until the coordinator starts a new instant + this.confirming = true; } @Override - public void snapshotState(FunctionSnapshotContext functionSnapshotContext) { + public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception { // Based on the fact that the coordinator starts the checkpoint first, // it would check the validity. // wait for the buffer data flush out and request a new instant flushRemaining(false); + // Reload the snapshot state as the current state. + reloadWriteMetaState(); } @Override @@ -216,6 +243,7 @@ public void close() { public void endInput() { flushRemaining(true); this.writeClient.cleanHandles(); + this.writeStatuses.clear(); } // ------------------------------------------------------------------------- @@ -275,6 +303,49 @@ private void initWriteFunction() { } } + private void restoreWriteMetadata() throws Exception { + String lastInflight = this.writeClient.getLastPendingInstant(this.actionType); + boolean eventSent = false; + for (WriteMetadataEvent event : this.writeMetadataState.get()) { + if (Objects.equals(lastInflight, event.getInstantTime())) { + // The checkpoint succeed but the meta does not commit, + // re-commit the inflight instant + this.eventGateway.sendEventToCoordinator(event); + LOG.info("Send uncommitted write metadata event to coordinator, task[{}].", taskID); + eventSent = true; + } + } + if (!eventSent) { + sendBootstrapEvent(); + } + } + + private void sendBootstrapEvent() { + WriteMetadataEvent event = WriteMetadataEvent.builder() + .taskID(taskID) + .writeStatus(Collections.emptyList()) + .instantTime("") + .isBootstrap(true) + .build(); + this.eventGateway.sendEventToCoordinator(event); + LOG.info("Send bootstrap write metadata event to coordinator, task[{}].", taskID); + } + + /** + * Reload the write metadata state as the current checkpoint. + */ + private void reloadWriteMetaState() throws Exception { + this.writeMetadataState.clear(); + WriteMetadataEvent event = WriteMetadataEvent.builder() + .taskID(taskID) + .instantTime(currentInstant) + .writeStatus(new ArrayList<>(writeStatuses)) + .isBootstrap(true) + .build(); + this.writeMetadataState.add(event); + writeStatuses.clear(); + } + /** * Represents a data item in the buffer, this is needed to reduce the * memory footprint. @@ -478,23 +549,23 @@ private void bufferRecord(HoodieRecord value) { bucket.records.add(item); } - @SuppressWarnings("unchecked, rawtypes") - private boolean flushBucket(DataBucket bucket) { - String instant = this.writeClient.getLastPendingInstant(this.actionType); - - if (instant == null) { - // in case there are empty checkpoints that has no input data - LOG.info("No inflight instant when flushing data, skip."); - return false; - } + private boolean hasData() { + return this.buckets.size() > 0 + && this.buckets.values().stream().anyMatch(bucket -> bucket.records.size() > 0); + } + private String instantToWrite(boolean hasData) { + String instant = this.writeClient.getLastPendingInstant(this.actionType); // if exactly-once semantics turns on, // waits for the checkpoint notification until the checkpoint timeout threshold hits. if (confirming) { long waitingTime = 0L; long ckpTimeout = config.getLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT); long interval = 500L; - while (Objects.equals(instant, this.currentInstant)) { + // wait condition: + // 1. there is no inflight instant + // 2. the inflight instant does not change and the checkpoint has buffering data + while (instant == null || (instant.equals(this.currentInstant) && hasData)) { // sleep for a while try { if (waitingTime > ckpTimeout) { @@ -512,6 +583,18 @@ private boolean flushBucket(DataBucket bucket) { // successfully. confirming = false; } + return instant; + } + + @SuppressWarnings("unchecked, rawtypes") + private boolean flushBucket(DataBucket bucket) { + String instant = instantToWrite(true); + + if (instant == null) { + // in case there are empty checkpoints that has no input data + LOG.info("No inflight instant when flushing data, skip."); + return false; + } List records = bucket.writeBuffer(); ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records"); @@ -521,20 +604,22 @@ private boolean flushBucket(DataBucket bucket) { bucket.preWrite(records); final List writeStatus = new ArrayList<>(writeFunction.apply(records, instant)); records.clear(); - final BatchWriteSuccessEvent event = BatchWriteSuccessEvent.builder() + final WriteMetadataEvent event = WriteMetadataEvent.builder() .taskID(taskID) .instantTime(instant) // the write instant may shift but the event still use the currentInstant. .writeStatus(writeStatus) .isLastBatch(false) .isEndInput(false) .build(); + this.eventGateway.sendEventToCoordinator(event); + writeStatuses.addAll(writeStatus); return true; } @SuppressWarnings("unchecked, rawtypes") private void flushRemaining(boolean isEndInput) { - this.currentInstant = this.writeClient.getLastPendingInstant(this.actionType); + this.currentInstant = instantToWrite(hasData()); if (this.currentInstant == null) { // in case there are empty checkpoints that has no input data throw new HoodieException("No inflight instant when flushing data!"); @@ -561,17 +646,20 @@ private void flushRemaining(boolean isEndInput) { LOG.info("No data to write in subtask [{}] for instant [{}]", taskID, currentInstant); writeStatus = Collections.emptyList(); } - final BatchWriteSuccessEvent event = BatchWriteSuccessEvent.builder() + final WriteMetadataEvent event = WriteMetadataEvent.builder() .taskID(taskID) .instantTime(currentInstant) .writeStatus(writeStatus) .isLastBatch(true) .isEndInput(isEndInput) .build(); + this.eventGateway.sendEventToCoordinator(event); this.buckets.clear(); this.tracer.reset(); this.writeClient.cleanHandles(); + this.writeStatuses.addAll(writeStatus); + // blocks flushing until the coordinator starts a new instant this.confirming = true; } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java index 3150d065a4026..b0f8328c1d56c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperator.java @@ -51,7 +51,7 @@ void setOperatorEventGateway(OperatorEventGateway operatorEventGateway) { } @Override - public void endInput() throws Exception { + public void endInput() { sinkFunction.endInput(); } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 7f6f8163c6534..84f3c0b1e4b98 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sink.event.BatchWriteSuccessEvent; +import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.utils.CoordinatorExecutor; import org.apache.hudi.sink.utils.HiveSyncContext; import org.apache.hudi.sink.utils.NonThrownExecutor; @@ -97,7 +97,7 @@ public class StreamWriteOperatorCoordinator * Event buffer for one round of checkpointing. When all the elements are non-null and have the same * write instant, then the instant succeed and we can commit it. */ - private transient BatchWriteSuccessEvent[] eventBuffer; + private transient WriteMetadataEvent[] eventBuffer; /** * Task number of the operator. @@ -152,8 +152,6 @@ public void start() throws Exception { this.tableState = TableState.create(conf); // init table, create it if not exists. initTableIfNotExists(this.conf); - // start a new instant - startInstant(); // start the executor this.executor = new CoordinatorExecutor(this.context, LOG); // start the executor if required @@ -201,7 +199,7 @@ public void notifyCheckpointComplete(long checkpointId) { // for streaming mode, commits the ever received events anyway, // the stream write task snapshot and flush the data buffer synchronously in sequence, // so a successful checkpoint subsumes the old one(follows the checkpoint subsuming contract) - final boolean committed = commitInstant(); + final boolean committed = commitInstant(this.instant); if (committed) { // if async compaction is on, schedule the compaction if (asyncCompaction) { @@ -216,30 +214,8 @@ public void notifyCheckpointComplete(long checkpointId) { ); } - private void syncHiveIfEnabled() { - if (conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED)) { - this.hiveSyncExecutor.execute(this::syncHive, "sync hive metadata for instant %s", this.instant); - } - } - - /** - * Sync hoodie table metadata to Hive metastore. - */ - public void syncHive() { - hiveSyncContext.hiveSyncTool().syncHoodieTable(); - } - - private void startInstant() { - final String instant = HoodieActiveTimeline.createNewInstantTime(); - this.writeClient.startCommitWithTime(instant, tableState.commitAction); - this.instant = instant; - this.writeClient.transitionRequestedToInflight(tableState.commitAction, this.instant); - LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant, - this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE)); - } - @Override - public void resetToCheckpoint(long checkpointID, @Nullable byte[] checkpointData) { + public void resetToCheckpoint(long checkpointID, byte[] checkpointData) { // no operation } @@ -248,27 +224,17 @@ public void handleEventFromOperator(int i, OperatorEvent operatorEvent) { executor.execute( () -> { // no event to handle - ValidationUtils.checkState(operatorEvent instanceof BatchWriteSuccessEvent, - "The coordinator can only handle BatchWriteSuccessEvent"); - BatchWriteSuccessEvent event = (BatchWriteSuccessEvent) operatorEvent; - // the write task does not block after checkpointing(and before it receives a checkpoint success event), - // if it checkpoints succeed then flushes the data buffer again before this coordinator receives a checkpoint - // success event, the data buffer would flush with an older instant time. - ValidationUtils.checkState( - HoodieTimeline.compareTimestamps(instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, event.getInstantTime()), - String.format("Receive an unexpected event for instant %s from task %d", - event.getInstantTime(), event.getTaskID())); - if (this.eventBuffer[event.getTaskID()] != null) { - this.eventBuffer[event.getTaskID()].mergeWith(event); + ValidationUtils.checkState(operatorEvent instanceof WriteMetadataEvent, + "The coordinator can only handle WriteMetaEvent"); + WriteMetadataEvent event = (WriteMetadataEvent) operatorEvent; + if (event.isBootstrap()) { + handleBootstrapEvent(event); + } else if (event.isEndInput()) { + handleEndInputEvent(event); } else { - this.eventBuffer[event.getTaskID()] = event; + handleWriteMetaEvent(event); } - if (event.isEndInput() && allEventsReceived()) { - // start to commit the instant. - commitInstant(); - // no compaction scheduling for batch mode - } - }, "handle write success event for instant %s", this.instant + }, "handle write metadata event for instant %s", this.instant ); } @@ -291,22 +257,108 @@ private void initHiveSync() { this.hiveSyncContext = HiveSyncContext.create(conf); } + private void syncHiveIfEnabled() { + if (conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED)) { + this.hiveSyncExecutor.execute(this::syncHive, "sync hive metadata for instant %s", this.instant); + } + } + + /** + * Sync hoodie table metadata to Hive metastore. + */ + public void syncHive() { + hiveSyncContext.hiveSyncTool().syncHoodieTable(); + } + private void reset() { - this.eventBuffer = new BatchWriteSuccessEvent[this.parallelism]; + this.eventBuffer = new WriteMetadataEvent[this.parallelism]; } - /** Checks the buffer is ready to commit. */ + /** + * Checks the buffer is ready to commit. + */ private boolean allEventsReceived() { return Arrays.stream(eventBuffer) .allMatch(event -> event != null && event.isReady(this.instant)); } + private void addEventToBuffer(WriteMetadataEvent event) { + if (this.eventBuffer[event.getTaskID()] != null) { + this.eventBuffer[event.getTaskID()].mergeWith(event); + } else { + this.eventBuffer[event.getTaskID()] = event; + } + } + + private void startInstant() { + final String instant = HoodieActiveTimeline.createNewInstantTime(); + this.writeClient.startCommitWithTime(instant, tableState.commitAction); + this.instant = instant; + this.writeClient.transitionRequestedToInflight(tableState.commitAction, this.instant); + LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant, + this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE)); + } + + /** + * Initializes the instant. + * + *

Recommits the last inflight instant if the write metadata checkpoint successfully + * but was not committed due to some rare cases. + * + *

Starts a new instant, a writer can not flush data buffer + * until it finds a new inflight instant on the timeline. + */ + private void initInstant(String instant) { + HoodieTimeline completedTimeline = + StreamerUtil.createMetaClient(conf).getActiveTimeline().filterCompletedInstants(); + executor.execute(() -> { + if (instant.equals("") || completedTimeline.containsInstant(instant)) { + // the last instant committed successfully + reset(); + } else { + LOG.info("Recommit instant {}", instant); + commitInstant(instant); + } + // starts a new instant + startInstant(); + }, "initialize instant %s", instant); + } + + private void handleBootstrapEvent(WriteMetadataEvent event) { + addEventToBuffer(event); + if (Arrays.stream(eventBuffer).allMatch(Objects::nonNull)) { + // start to initialize the instant. + initInstant(event.getInstantTime()); + } + } + + private void handleEndInputEvent(WriteMetadataEvent event) { + addEventToBuffer(event); + if (allEventsReceived()) { + // start to commit the instant. + commitInstant(this.instant); + // no compaction scheduling for batch mode + } + } + + private void handleWriteMetaEvent(WriteMetadataEvent event) { + // the write task does not block after checkpointing(and before it receives a checkpoint success event), + // if it checkpoints succeed then flushes the data buffer again before this coordinator receives a checkpoint + // success event, the data buffer would flush with an older instant time. + ValidationUtils.checkState( + HoodieTimeline.compareTimestamps(this.instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, event.getInstantTime()), + String.format("Receive an unexpected event for instant %s from task %d", + event.getInstantTime(), event.getTaskID())); + + addEventToBuffer(event); + } + /** * Commits the instant. * * @return true if the write statuses are committed successfully. */ - private boolean commitInstant() { + private boolean commitInstant(String instant) { if (Arrays.stream(eventBuffer).allMatch(Objects::isNull)) { // The last checkpoint finished successfully. return false; @@ -314,7 +366,7 @@ private boolean commitInstant() { List writeResults = Arrays.stream(eventBuffer) .filter(Objects::nonNull) - .map(BatchWriteSuccessEvent::getWriteStatuses) + .map(WriteMetadataEvent::getWriteStatuses) .flatMap(Collection::stream) .collect(Collectors.toList()); @@ -323,13 +375,15 @@ private boolean commitInstant() { reset(); return false; } - doCommit(writeResults); + doCommit(instant, writeResults); return true; } - /** Performs the actual commit action. */ + /** + * Performs the actual commit action. + */ @SuppressWarnings("unchecked") - private void doCommit(List writeResults) { + private void doCommit(String instant, List writeResults) { // commit or rollback long totalErrorRecords = writeResults.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L); long totalRecords = writeResults.stream().map(WriteStatus::getTotalRecords).reduce(Long::sum).orElse(0L); @@ -345,13 +399,13 @@ private void doCommit(List writeResults) { final Map> partitionToReplacedFileIds = tableState.isOverwrite ? writeClient.getPartitionToReplacedFileIds(tableState.operationType, writeResults) : Collections.emptyMap(); - boolean success = writeClient.commit(this.instant, writeResults, Option.of(checkpointCommitMetadata), + boolean success = writeClient.commit(instant, writeResults, Option.of(checkpointCommitMetadata), tableState.commitAction, partitionToReplacedFileIds); if (success) { reset(); - LOG.info("Commit instant [{}] success!", this.instant); + LOG.info("Commit instant [{}] success!", instant); } else { - throw new HoodieException(String.format("Commit instant [%s] failed!", this.instant)); + throw new HoodieException(String.format("Commit instant [%s] failed!", instant)); } } else { LOG.error("Error when writing. Errors/Total=" + totalErrorRecords + "/" + totalRecords); @@ -364,13 +418,13 @@ private void doCommit(List writeResults) { } }); // Rolls back instant - writeClient.rollback(this.instant); - throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", this.instant)); + writeClient.rollback(instant); + throw new HoodieException(String.format("Commit instant [%s] failed and rolled back !", instant)); } } @VisibleForTesting - public BatchWriteSuccessEvent[] getEventBuffer() { + public WriteMetadataEvent[] getEventBuffer() { return eventBuffer; } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java index 503a5bf06f1e7..4bb703e10e9f3 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapFunction.java @@ -22,20 +22,24 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.util.StreamerUtil; +import org.apache.avro.Schema; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; @@ -54,11 +58,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import static java.util.stream.Collectors.toList; + /** * The function to load index from existing hoodieTable. * @@ -78,6 +83,7 @@ public class BootstrapFunction private final Configuration conf; private transient org.apache.hadoop.conf.Configuration hadoopConf; + private transient HoodieWriteConfig writeConfig; private GlobalAggregateManager aggregateManager; private ListState bootstrapState; @@ -108,13 +114,14 @@ public void initializeState(FunctionInitializationContext context) throws Except public void open(Configuration parameters) throws Exception { super.open(parameters); this.hadoopConf = StreamerUtil.getHadoopConf(); + this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf); this.hoodieTable = getTable(); this.aggregateManager = ((StreamingRuntimeContext) getRuntimeContext()).getGlobalAggregateManager(); } @Override @SuppressWarnings("unchecked") - public void processElement(I value, Context ctx, Collector out) throws IOException { + public void processElement(I value, Context ctx, Collector out) throws Exception { if (!alreadyBootstrap) { String basePath = hoodieTable.getMetaClient().getBasePath(); int taskID = getRuntimeContext().getIndexOfThisSubtask(); @@ -155,11 +162,10 @@ private void waitForBootstrapReady(int taskID) { } private HoodieFlinkTable getTable() { - HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf); HoodieFlinkEngineContext context = new HoodieFlinkEngineContext( new SerializableConfiguration(this.hadoopConf), new FlinkTaskContextSupplier(getRuntimeContext())); - return HoodieFlinkTable.create(writeConfig, context); + return HoodieFlinkTable.create(this.writeConfig, context); } /** @@ -168,32 +174,64 @@ private HoodieFlinkTable getTable() { * @param partitionPath The partition path */ @SuppressWarnings("unchecked") - private void loadRecords(String partitionPath, Collector out) { + private void loadRecords(String partitionPath, Collector out) throws Exception { long start = System.currentTimeMillis(); + BaseFileUtils fileUtils = BaseFileUtils.getInstance(this.hoodieTable.getBaseFileFormat()); - List latestBaseFiles = - HoodieIndexUtils.getLatestBaseFilesForPartition(partitionPath, this.hoodieTable); - LOG.info("All baseFile in partition {} size = {}", partitionPath, latestBaseFiles.size()); + Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema(); final int parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); final int maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks(); final int taskID = getRuntimeContext().getIndexOfThisSubtask(); - for (HoodieBaseFile baseFile : latestBaseFiles) { - boolean shouldLoad = KeyGroupRangeAssignment.assignKeyToParallelOperator( - baseFile.getFileId(), maxParallelism, parallelism) == taskID; - if (shouldLoad) { - LOG.info("Load records from file {}.", baseFile); - final List hoodieKeys; - try { - hoodieKeys = - fileUtils.fetchRecordKeyPartitionPath(this.hadoopConf, new Path(baseFile.getPath())); - } catch (Exception e) { - throw new HoodieException(String.format("Error when loading record keys from file: %s", baseFile), e); + Option latestCommitTime = this.hoodieTable.getMetaClient().getCommitsTimeline() + .filterCompletedInstants().lastInstant(); + + if (latestCommitTime.isPresent()) { + List fileSlices = this.hoodieTable.getSliceView() + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp(), true) + .collect(toList()); + + for (FileSlice fileSlice : fileSlices) { + if (!shouldLoadFile(fileSlice.getFileId(), maxParallelism, parallelism, taskID)) { + continue; } + LOG.info("Load records from {}.", fileSlice); + + // load parquet records + fileSlice.getBaseFile().ifPresent(baseFile -> { + // filter out crushed files + if (baseFile.getFileSize() <= 0) { + return; + } + + final List hoodieKeys; + try { + hoodieKeys = + fileUtils.fetchRecordKeyPartitionPath(this.hadoopConf, new Path(baseFile.getPath())); + } catch (Exception e) { + throw new HoodieException(String.format("Error when loading record keys from file: %s", baseFile), e); + } + + for (HoodieKey hoodieKey : hoodieKeys) { + out.collect((O) new IndexRecord(generateHoodieRecord(hoodieKey, fileSlice))); + } + }); + + // load avro log records + List logPaths = fileSlice.getLogFiles() + // filter out crushed files + .filter(logFile -> logFile.getFileSize() > 0) + .map(logFile -> logFile.getPath().toString()) + .collect(toList()); + HoodieMergedLogRecordScanner scanner = scanLog(logPaths, schema, latestCommitTime.get().getTimestamp()); - for (HoodieKey hoodieKey : hoodieKeys) { - out.collect((O) new IndexRecord(generateHoodieRecord(hoodieKey, baseFile))); + try { + for (String recordKey : scanner.getRecords().keySet()) { + out.collect((O) new IndexRecord(generateHoodieRecord(new HoodieKey(recordKey, partitionPath), fileSlice))); + } + } catch (Exception e) { + throw new HoodieException(String.format("Error when loading record keys from files: %s", logPaths), e); } } } @@ -203,17 +241,44 @@ private void loadRecords(String partitionPath, Collector out) { this.getClass().getSimpleName(), taskID, partitionPath, cost); } + private HoodieMergedLogRecordScanner scanLog( + List logPaths, + Schema logSchema, + String latestInstantTime) { + String basePath = this.hoodieTable.getMetaClient().getBasePath(); + return HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(FSUtils.getFs(basePath, this.hadoopConf)) + .withBasePath(basePath) + .withLogFilePaths(logPaths) + .withReaderSchema(logSchema) + .withLatestInstantTime(latestInstantTime) + .withReadBlocksLazily(this.writeConfig.getCompactionLazyBlockReadEnabled()) + .withReverseReader(false) + .withBufferSize(this.writeConfig.getMaxDFSStreamBufferSize()) + .withMaxMemorySizeInBytes(this.writeConfig.getMaxMemoryPerPartitionMerge()) + .withSpillableMapBasePath(this.writeConfig.getSpillableMapBasePath()) + .build(); + } + @SuppressWarnings("unchecked") - public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, HoodieBaseFile baseFile) { + public static HoodieRecord generateHoodieRecord(HoodieKey hoodieKey, FileSlice fileSlice) { HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, null); - hoodieRecord.setCurrentLocation(new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), baseFile.getCommitTime(), baseFile.getFileId())); + hoodieRecord.setCurrentLocation(new HoodieRecordGlobalLocation(hoodieKey.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId())); hoodieRecord.seal(); return hoodieRecord; } + private static boolean shouldLoadFile(String fileId, + int maxParallelism, + int parallelism, + int taskID) { + return KeyGroupRangeAssignment.assignKeyToParallelOperator( + fileId, maxParallelism, parallelism) == taskID; + } + @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { + public void notifyCheckpointComplete(long checkpointId) { // no operation } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java index ad3dd577a69e4..e271c84d32681 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.table.HoodieFlinkTable; +import org.apache.hudi.util.CompactionUtil; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.annotation.VisibleForTesting; @@ -80,18 +81,18 @@ public void processElement(StreamRecord streamRecord) { } @Override - public void notifyCheckpointComplete(long checkpointId) throws IOException { + public void notifyCheckpointComplete(long checkpointId) { try { - scheduleCompaction(checkpointId); + HoodieFlinkTable hoodieTable = writeClient.getHoodieTable(); + CompactionUtil.rollbackCompaction(hoodieTable, conf); + scheduleCompaction(hoodieTable, checkpointId); } catch (Throwable throwable) { // make it fail safe LOG.error("Error while scheduling compaction at instant: " + compactionInstantTime, throwable); } } - private void scheduleCompaction(long checkpointId) throws IOException { - HoodieFlinkTable table = writeClient.getHoodieTable(); - + private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) throws IOException { // the last instant takes the highest priority. Option lastRequested = table.getActiveTimeline().filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).lastInstant(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java index ddc73a59474cd..699f078a700d5 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java @@ -83,6 +83,12 @@ public class FlinkCompactionConfig extends Configuration { @Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB.", required = false) public Integer compactionMaxMemory = 100; + @Parameter(names = {"--compaction-target-io"}, description = "Target IO per compaction (both read and write) for batching compaction, default 512000M.", required = false) + public Long compactionTargetIo = 512000L; + + @Parameter(names = {"--compaction-tasks"}, description = "Parallelism of tasks that do actual compaction, default is 10", required = false) + public Integer compactionTasks = 10; + /** * Transforms a {@code HoodieFlinkCompaction.config} into {@code Configuration}. * The latter is more suitable for the table APIs. It reads all the properties @@ -100,6 +106,8 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkCo conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, config.compactionDeltaCommits); conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, config.compactionDeltaSeconds); conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory); + conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo); + conf.setInteger(FlinkOptions.COMPACTION_TASKS, config.compactionTasks); conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable); // use synchronous compaction always conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index 0ba2351b96181..3e0a4375b7efd 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -58,7 +58,7 @@ public static void main(String[] args) throws Exception { Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg); // create metaClient - HoodieTableMetaClient metaClient = CompactionUtil.createMetaClient(conf); + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); // get the table name conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); @@ -66,17 +66,24 @@ public static void main(String[] args) throws Exception { // set table schema CompactionUtil.setAvroSchema(conf, metaClient); + HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + // rolls back inflight compaction first + // condition: the schedule compaction is in INFLIGHT state for max delta seconds. + CompactionUtil.rollbackCompaction(table, conf); + // judge whether have operation // to compute the compaction instant time and do compaction. String compactionInstantTime = CompactionUtil.getCompactionInstantTime(metaClient); - HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null); boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); if (!scheduled) { // do nothing. LOG.info("No compaction plan for this job "); return; } - HoodieFlinkTable table = writeClient.getHoodieTable(); + + table.getMetaClient().reloadActiveTimeline(); // generate compaction plan // should support configurable commit metadata HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( @@ -104,6 +111,9 @@ public static void main(String[] args) throws Exception { return; } + // get compactionParallelism. + int compactionParallelism = Math.min(conf.getInteger(FlinkOptions.COMPACTION_TASKS), compactionPlan.getOperations().size()); + env.addSource(new CompactionPlanSourceFunction(table, instant, compactionPlan, compactionInstantTime)) .name("compaction_source") .uid("uid_compaction_source") @@ -111,7 +121,7 @@ public static void main(String[] args) throws Exception { .transform("compact_task", TypeInformation.of(CompactionCommitEvent.class), new ProcessOperator<>(new CompactFunction(conf))) - .setParallelism(compactionPlan.getOperations().size()) + .setParallelism(compactionParallelism) .addSink(new CompactionCommitSink(conf)) .name("clean_commits") .uid("uid_clean_commits") diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/event/BatchWriteSuccessEvent.java b/hudi-flink/src/main/java/org/apache/hudi/sink/event/WriteMetadataEvent.java similarity index 81% rename from hudi-flink/src/main/java/org/apache/hudi/sink/event/BatchWriteSuccessEvent.java rename to hudi-flink/src/main/java/org/apache/hudi/sink/event/WriteMetadataEvent.java index 186a470ea2967..662383b50950b 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/event/BatchWriteSuccessEvent.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/event/WriteMetadataEvent.java @@ -30,13 +30,14 @@ /** * An operator event to mark successful checkpoint batch write. */ -public class BatchWriteSuccessEvent implements OperatorEvent { +public class WriteMetadataEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private List writeStatuses; private final int taskID; private String instantTime; private boolean isLastBatch; + /** * Flag saying whether the event comes from the end of input, e.g. the source * is bounded, there are two cases in which this flag should be set to true: @@ -45,6 +46,11 @@ public class BatchWriteSuccessEvent implements OperatorEvent { */ private final boolean isEndInput; + /** + * Flag saying whether the event comes from bootstrap of a write function. + */ + private final boolean isBootstrap; + /** * Creates an event. * @@ -55,22 +61,25 @@ public class BatchWriteSuccessEvent implements OperatorEvent { * within an checkpoint interval, * if true, the whole data set of the checkpoint * has been flushed successfully + * @param isBootstrap Whether the event comes from the bootstrap */ - private BatchWriteSuccessEvent( + private WriteMetadataEvent( int taskID, String instantTime, List writeStatuses, boolean isLastBatch, - boolean isEndInput) { + boolean isEndInput, + boolean isBootstrap) { this.taskID = taskID; this.instantTime = instantTime; this.writeStatuses = new ArrayList<>(writeStatuses); this.isLastBatch = isLastBatch; this.isEndInput = isEndInput; + this.isBootstrap = isBootstrap; } /** - * Returns the builder for {@link BatchWriteSuccessEvent}. + * Returns the builder for {@link WriteMetadataEvent}. */ public static Builder builder() { return new Builder(); @@ -96,12 +105,16 @@ public boolean isEndInput() { return isEndInput; } + public boolean isBootstrap() { + return isBootstrap; + } + /** - * Merges this event with given {@link BatchWriteSuccessEvent} {@code other}. + * Merges this event with given {@link WriteMetadataEvent} {@code other}. * * @param other The event to be merged */ - public void mergeWith(BatchWriteSuccessEvent other) { + public void mergeWith(WriteMetadataEvent other) { ValidationUtils.checkArgument(this.taskID == other.taskID); // the instant time could be monotonically increasing this.instantTime = other.instantTime; @@ -112,7 +125,9 @@ public void mergeWith(BatchWriteSuccessEvent other) { this.writeStatuses = statusList; } - /** Returns whether the event is ready to commit. */ + /** + * Returns whether the event is ready to commit. + */ public boolean isReady(String currentInstant) { return isLastBatch && this.instantTime.equals(currentInstant); } @@ -122,7 +137,7 @@ public boolean isReady(String currentInstant) { // ------------------------------------------------------------------------- /** - * Builder for {@link BatchWriteSuccessEvent}. + * Builder for {@link WriteMetadataEvent}. */ public static class Builder { private List writeStatus; @@ -130,12 +145,13 @@ public static class Builder { private String instantTime; private boolean isLastBatch = false; private boolean isEndInput = false; + private boolean isBootstrap = false; - public BatchWriteSuccessEvent build() { + public WriteMetadataEvent build() { Objects.requireNonNull(taskID); Objects.requireNonNull(instantTime); Objects.requireNonNull(writeStatus); - return new BatchWriteSuccessEvent(taskID, instantTime, writeStatus, isLastBatch, isEndInput); + return new WriteMetadataEvent(taskID, instantTime, writeStatus, isLastBatch, isEndInput, isBootstrap); } public Builder taskID(int taskID) { @@ -162,5 +178,10 @@ public Builder isEndInput(boolean isEndInput) { this.isEndInput = isEndInput; return this; } + + public Builder isBootstrap(boolean isBootstrap) { + this.isBootstrap = isBootstrap; + return this; + } } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java index e2f32937b132d..6d805ce8d83ec 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketAssigner.java @@ -228,7 +228,7 @@ public boolean assign() { return false; } SmallFileAssignState state = states[assignIdx]; - if (!state.canAssign()) { + while (!state.canAssign()) { assignIdx += 1; if (assignIdx >= states.length) { noSpace = true; @@ -236,7 +236,6 @@ public boolean assign() { } // move to next slot if possible state = states[assignIdx]; - assert state.canAssign(); } state.assign(); return true; diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java index 3679c8a6602df..093cef5be2990 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java @@ -34,11 +34,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; /** @@ -119,12 +121,16 @@ private static List getWritePathsOfInstant(Path basePath, HoodieComm }).map(path -> { try { return fs.getFileStatus(path); + } catch (FileNotFoundException fe) { + LOG.warn("File {} was deleted by the cleaner, ignore", path); + return null; } catch (IOException e) { LOG.error("Get write status of path: {} error", path); throw new HoodieException(e); } }) // filter out crushed files + .filter(Objects::nonNull) .filter(fileStatus -> fileStatus.getLen() > 0) .collect(Collectors.toList()); } @@ -144,11 +150,14 @@ public static Option getCommitMetadataSafely( Path basePath, HoodieInstant instant, HoodieTimeline timeline) { - byte[] data = timeline.getInstantDetails(instant).get(); try { + byte[] data = timeline.getInstantDetails(instant).get(); return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class)); - } catch (IOException e) { + } catch (FileNotFoundException fe) { // make this fail safe. + LOG.warn("Instant {} was deleted by the cleaner, ignore", instant.getTimestamp()); + return Option.empty(); + } catch (Throwable throwable) { LOG.error("Get write metadata for table {} with instant {} and path: {} error", tableName, instant.getTimestamp(), basePath); return Option.empty(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index 62a67980d6373..db1d577c026c6 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -83,7 +83,7 @@ private static HiveSyncConfig buildSyncConfig(Configuration conf) { hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS); hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP); hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB); - hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.PARTITION_PATH_URL_ENCODE); + hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING); hiveSyncConfig.skipROSuffix = conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX); hiveSyncConfig.assumeDatePartitioning = conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION); return hiveSyncConfig; diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index 59534cf15a04a..73c3448c20241 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.apache.hudi.util.StreamerUtil; @@ -130,6 +131,155 @@ public class FlinkStreamerConfig extends Configuration { @Parameter(names = {"--write-task-num"}, description = "Parallelism of tasks that do actual write, default is 4.") public Integer writeTaskNum = 4; + @Parameter(names = {"--partition-default-name"}, + description = "The default partition name in case the dynamic partition column value is null/empty string") + public String partitionDefaultName = "__DEFAULT_PARTITION__"; + + @Parameter(names = {"--index-bootstrap-enabled"}, + description = "Whether to bootstrap the index state from existing hoodie table, default false") + public Boolean indexBootstrapEnabled = false; + + @Parameter(names = {"--index-state-ttl"}, description = "Index state ttl in days, default 1.5 day") + public Double indexStateTtl = 1.5D; + + @Parameter(names = {"--index-global-enabled"}, description = "Whether to update index for the old partition path " + + "if same key record with different partition path came in, default false") + public Boolean indexGlobalEnabled = false; + + @Parameter(names = {"--index-partition-regex"}, + description = "Whether to load partitions in state if partition path matching, default *") + public String indexPartitionRegex = ".*"; + + @Parameter(names = {"--avro-schema-path"}, description = "Avro schema file path, the parsed schema is used for deserialization") + public String avroSchemaPath = ""; + + @Parameter(names = {"--avro-schema"}, description = "Avro schema string, the parsed schema is used for deserialization") + public String avroSchema = ""; + + @Parameter(names = {"--utc-timezone"}, description = "Use UTC timezone or local timezone to the conversion between epoch" + + " time and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x" + + " use UTC timezone, by default true") + public Boolean utcTimezone = true; + + @Parameter(names = {"--write-partition-url-encode"}, description = "Whether to encode the partition path url, default false") + public Boolean writePartitionUrlEncode = false; + + @Parameter(names = {"--hive-style-partitioning"}, description = "Whether to use Hive style partitioning.\n" + + "If set true, the names of partition folders follow = format.\n" + + "By default false (the names of partition folders are only partition values)") + public Boolean hiveStylePartitioning = false; + + @Parameter(names = {"--write-task-max-size"}, description = "Maximum memory in MB for a write task, when the threshold hits,\n" + + "it flushes the max size data bucket to avoid OOM, default 1GB") + public Double writeTaskMaxSize = 1024D; + + @Parameter(names = {"--write-batch-size"}, + description = "Batch buffer size in MB to flush data into the underneath filesystem, default 64MB") + public Double writeBatchSize = 64D; + + @Parameter(names = {"--write-log-block-size"}, description = "Max log block size in MB for log file, default 128MB") + public Integer writeLogBlockSize = 128; + + @Parameter(names = {"--write-log-max-size"}, + description = "Maximum size allowed in MB for a log file before it is rolled over to the next version, default 1GB") + public Integer writeLogMaxSize = 1024; + + @Parameter(names = {"--write-merge-max-memory"}, description = "Max memory in MB for merge, default 100MB") + public Integer writeMergeMaxMemory = 100; + + @Parameter(names = {"--compaction-async-enabled"}, description = "Async Compaction, enabled by default for MOR") + public Boolean compactionAsyncEnabled = true; + + @Parameter(names = {"--compaction-tasks"}, description = "Parallelism of tasks that do actual compaction, default is 10") + public Integer compactionTasks = 10; + + @Parameter(names = {"--compaction-trigger-strategy"}, + description = "Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits;\n" + + "'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction;\n" + + "'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied;\n" + + "'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied.\n" + + "Default is 'num_commits'") + public String compactionTriggerStrategy = FlinkOptions.NUM_COMMITS; + + @Parameter(names = {"--compaction-delta-commits"}, description = "Max delta commits needed to trigger compaction, default 5 commits") + public Integer compactionDeltaCommits = 5; + + @Parameter(names = {"--compaction-delta-seconds"}, description = "Max delta seconds time needed to trigger compaction, default 1 hour") + public Integer compactionDeltaSeconds = 3600; + + @Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB") + public Integer compactionMaxMemory = 100; + + @Parameter(names = {"--compaction-target-io"}, description = "Target IO per compaction (both read and write), default 5 GB") + public Long compactionTargetIo = 5120L; + + @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default") + public Boolean cleanAsyncEnabled = true; + + @Parameter(names = {"--clean-retain-commits"}, + description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n" + + "This also directly translates into how much you can incrementally pull on this table, default 10") + public Integer cleanRetainCommits = 10; + + @Parameter(names = {"--archive-max-commits"}, + description = "Max number of commits to keep before archiving older commits into a sequential log, default 30") + public Integer archiveMaxCommits = 30; + + @Parameter(names = {"--archive-min-commits"}, + description = "Min number of commits to keep before archiving older commits into a sequential log, default 20") + public Integer archiveMinCommits = 20; + + @Parameter(names = {"--hive-sync-enable"}, description = "Asynchronously sync Hive meta to HMS, default false") + public Boolean hiveSyncEnabled = false; + + @Parameter(names = {"--hive-sync-db"}, description = "Database name for hive sync, default 'default'") + public String hiveSyncDb = "default"; + + @Parameter(names = {"--hive-sync-table"}, description = "Table name for hive sync, default 'unknown'") + public String hiveSyncTable = "unknown"; + + @Parameter(names = {"--hive-sync-file-format"}, description = "File format for hive sync, default 'PARQUET'") + public String hiveSyncFileFormat = "PARQUET"; + + @Parameter(names = {"--hive-sync-username"}, description = "Username for hive sync, default 'hive'") + public String hiveSyncUsername = "hive"; + + @Parameter(names = {"--hive-sync-password"}, description = "Password for hive sync, default 'hive'") + public String hiveSyncPassword = "hive"; + + @Parameter(names = {"--hive-sync-jdbc-url"}, description = "Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'") + public String hiveSyncJdbcUrl = "jdbc:hive2://localhost:10000"; + + @Parameter(names = {"--hive-sync-metastore-uris"}, description = "Metastore uris for hive sync, default ''") + public String hiveSyncMetastoreUri = ""; + + @Parameter(names = {"--hive-sync-partition-fields"}, description = "Partition fields for hive sync, default ''") + public String hiveSyncPartitionFields = ""; + + @Parameter(names = {"--hive-sync-partition-extractor-class"}, description = "Tool to extract the partition value from HDFS path, " + + "default 'SlashEncodedDayPartitionValueExtractor'") + public String hiveSyncPartitionExtractorClass = SlashEncodedDayPartitionValueExtractor.class.getCanonicalName(); + + @Parameter(names = {"--hive-sync-assume-date-partitioning"}, description = "Assume partitioning is yyyy/mm/dd, default false") + public Boolean hiveSyncAssumeDatePartition = false; + + @Parameter(names = {"--hive-sync-use-jdbc"}, description = "Use JDBC when hive synchronization is enabled, default true") + public Boolean hiveSyncUseJdbc = true; + + @Parameter(names = {"--hive-sync-auto-create-db"}, description = "Auto create hive database if it does not exists, default true") + public Boolean hiveSyncAutoCreateDb = true; + + @Parameter(names = {"--hive-sync-ignore-exceptions"}, description = "Ignore exceptions during hive synchronization, default false") + public Boolean hiveSyncIgnoreExceptions = false; + + @Parameter(names = {"--hive-sync-skip-ro-suffix"}, description = "Skip the _ro suffix for Read optimized table when registering, default false") + public Boolean hiveSyncSkipRoSuffix = false; + + @Parameter(names = {"--hive-sync-support-timestamp"}, description = "INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type.\n" + + "Disabled by default for backward compatibility.") + public Boolean hiveSyncSupportTimestamp = false; + + /** * Transforms a {@code HoodieFlinkStreamer.Config} into {@code Configuration}. * The latter is more suitable for the table APIs. It reads all the properties @@ -155,14 +305,54 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setBoolean(FlinkOptions.IGNORE_FAILED, config.commitOnErrors); conf.setString(FlinkOptions.RECORD_KEY_FIELD, config.recordKeyField); conf.setString(FlinkOptions.PARTITION_PATH_FIELD, config.partitionPathField); - conf.setString(FlinkOptions.KEYGEN_CLASS, config.keygenClass); if (!StringUtils.isNullOrEmpty(config.keygenClass)) { conf.setString(FlinkOptions.KEYGEN_CLASS, config.keygenClass); } else { conf.setString(FlinkOptions.KEYGEN_TYPE, config.keygenType); } conf.setInteger(FlinkOptions.WRITE_TASKS, config.writeTaskNum); - + conf.setString(FlinkOptions.PARTITION_DEFAULT_NAME, config.partitionDefaultName); + conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, config.indexBootstrapEnabled); + conf.setDouble(FlinkOptions.INDEX_STATE_TTL, config.indexStateTtl); + conf.setBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED, config.indexGlobalEnabled); + conf.setString(FlinkOptions.INDEX_PARTITION_REGEX, config.indexPartitionRegex); + conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, config.avroSchemaPath); + conf.setString(FlinkOptions.READ_AVRO_SCHEMA, config.avroSchema); + conf.setBoolean(FlinkOptions.UTC_TIMEZONE, config.utcTimezone); + conf.setBoolean(FlinkOptions.URL_ENCODE_PARTITIONING, config.writePartitionUrlEncode); + conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, config.hiveStylePartitioning); + conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, config.writeTaskMaxSize); + conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, config.writeBatchSize); + conf.setInteger(FlinkOptions.WRITE_LOG_BLOCK_SIZE, config.writeLogBlockSize); + conf.setInteger(FlinkOptions.WRITE_LOG_MAX_SIZE, config.writeLogMaxSize); + conf.setInteger(FlinkOptions.WRITE_MERGE_MAX_MEMORY, config.writeMergeMaxMemory); + conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, config.compactionAsyncEnabled); + conf.setInteger(FlinkOptions.COMPACTION_TASKS, config.compactionTasks); + conf.setString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY, config.compactionTriggerStrategy); + conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, config.compactionDeltaCommits); + conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, config.compactionDeltaSeconds); + conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory); + conf.setLong(FlinkOptions.COMPACTION_TARGET_IO, config.compactionTargetIo); + conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnabled); + conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits); + conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits); + conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits); + conf.setBoolean(FlinkOptions.HIVE_SYNC_ENABLED, config.hiveSyncEnabled); + conf.setString(FlinkOptions.HIVE_SYNC_DB, config.hiveSyncDb); + conf.setString(FlinkOptions.HIVE_SYNC_TABLE, config.hiveSyncTable); + conf.setString(FlinkOptions.HIVE_SYNC_FILE_FORMAT, config.hiveSyncFileFormat); + conf.setString(FlinkOptions.HIVE_SYNC_USERNAME, config.hiveSyncUsername); + conf.setString(FlinkOptions.HIVE_SYNC_PASSWORD, config.hiveSyncPassword); + conf.setString(FlinkOptions.HIVE_SYNC_JDBC_URL, config.hiveSyncJdbcUrl); + conf.setString(FlinkOptions.HIVE_SYNC_METASTORE_URIS, config.hiveSyncMetastoreUri); + conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS, config.hiveSyncPartitionFields); + conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS, config.hiveSyncPartitionExtractorClass); + conf.setBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION, config.hiveSyncAssumeDatePartition); + conf.setBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC, config.hiveSyncUseJdbc); + conf.setBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB, config.hiveSyncAutoCreateDb); + conf.setBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS, config.hiveSyncIgnoreExceptions); + conf.setBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX, config.hiveSyncSkipRoSuffix); + conf.setBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP, config.hiveSyncSupportTimestamp); return conf; } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java index bee5fe641aecb..cc5f6c030b345 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java +++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/HoodieFlinkStreamer.java @@ -99,7 +99,7 @@ public static void main(String[] args) throws Exception { .uid("uid_kafka_source") .map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class)); if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { - hoodieDataStream = hoodieDataStream.transform("index_bootstrap", + hoodieDataStream = hoodieDataStream.rebalance().transform("index_bootstrap", TypeInformation.of(HoodieRecord.class), new ProcessOperator<>(new BootstrapFunction<>(conf))); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java index ae25644d19450..a8e38a4aa86e9 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSink.java @@ -80,7 +80,7 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { // TODO: This is a very time-consuming operation, will optimization if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) { - hoodieDataStream = hoodieDataStream.transform("index_bootstrap", + hoodieDataStream = hoodieDataStream.rebalance().transform("index_bootstrap", TypeInformation.of(HoodieRecord.class), new ProcessOperator<>(new BootstrapFunction<>(conf))); } @@ -96,13 +96,11 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { // shuffle by fileId(bucket id) .keyBy(record -> record.getCurrentLocation().getFileId()) .transform("hoodie_stream_write", TypeInformation.of(Object.class), operatorFactory) - .name("uid_hoodie_stream_write") .setParallelism(numWriteTasks); if (StreamerUtil.needsAsyncCompaction(conf)) { return pipeline.transform("compact_plan_generate", TypeInformation.of(CompactionPlanEvent.class), new CompactionPlanOperator(conf)) - .name("uid_compact_plan_generate") .setParallelism(1) // plan generate must be singleton .rebalance() .transform("compact_task", diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index 55ec46a2ebcb2..786023efa335f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -218,7 +218,7 @@ public Result applyFilters(List filters) { @Override public Optional>> listPartitions() { List> partitions = FilePathUtils.getPartitions(path, hadoopConf, - partitionKeys, defaultPartName, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION)); + partitionKeys, defaultPartName, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING)); return Optional.of(partitions); } @@ -446,7 +446,7 @@ public Path[] getReadPaths() { return partitionKeys.isEmpty() ? new Path[] {path} : FilePathUtils.partitionPath2ReadPath(path, partitionKeys, getOrFetchPartitions(), - conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION)); + conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING)); } private static class LatestFileFilter extends FilePathFilter { diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java index e8ee5a244f15e..83607cd9c714d 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java @@ -347,7 +347,7 @@ public static Path[] getReadPaths( return new Path[] {path}; } else { final String defaultParName = conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME); - final boolean hivePartition = conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION); + final boolean hivePartition = conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING); List> partitionPaths = getPartitions(path, hadoopConf, partitionKeys, defaultParName, hivePartition); return partitionPath2ReadPath(path, partitionKeys, partitionPaths, hivePartition); diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index c7a910608d075..aa7453cd9d4d4 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -272,7 +272,7 @@ private ParquetColumnarRowSplitReader getReader(String path, int[] requiredPos) // generate partition specs. LinkedHashMap partSpec = FilePathUtils.extractPartitionKeyValues( new org.apache.hadoop.fs.Path(path).getParent(), - this.conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION), + this.conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), FilePathUtils.extractPartitionKeys(this.conf)); LinkedHashMap partObjects = new LinkedHashMap<>(); partSpec.forEach((k, v) -> partObjects.put(k, restorePartValueFromType( @@ -572,13 +572,13 @@ static class MergeIterator implements RecordIterator { @Override public boolean reachedEnd() throws IOException { - if (!readLogs && !this.reader.reachedEnd()) { + while (!readLogs && !this.reader.reachedEnd()) { currentRecord = this.reader.nextRecord(); if (instantRange != null) { boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString()); if (!isInRange) { // filter base file by instant range - return reachedEnd(); + continue; } } final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString(); @@ -587,7 +587,7 @@ public boolean reachedEnd() throws IOException { Option mergedAvroRecord = mergeRowWithLog(currentRecord, curKey); if (!mergedAvroRecord.isPresent()) { // deleted - return reachedEnd(); + continue; } else { GenericRecord record = buildAvroRecordBySchema( mergedAvroRecord.get(), @@ -601,27 +601,27 @@ public boolean reachedEnd() throws IOException { // project the full record in base with required positions currentRecord = projection.project(currentRecord); return false; - } else { - readLogs = true; - while (logKeysIterator.hasNext()) { - final String curKey = logKeysIterator.next(); - if (!keyToSkip.contains(curKey)) { - Option insertAvroRecord = - logRecords.get(curKey).getData().getInsertValue(tableSchema); - if (insertAvroRecord.isPresent()) { - // the record is a DELETE if insertAvroRecord not present, skipping - GenericRecord requiredAvroRecord = buildAvroRecordBySchema( - insertAvroRecord.get(), - requiredSchema, - requiredPos, - recordBuilder); - this.currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord); - return false; - } + } + // read the logs + readLogs = true; + while (logKeysIterator.hasNext()) { + final String curKey = logKeysIterator.next(); + if (!keyToSkip.contains(curKey)) { + Option insertAvroRecord = + logRecords.get(curKey).getData().getInsertValue(tableSchema); + if (insertAvroRecord.isPresent()) { + // the record is a DELETE if insertAvroRecord not present, skipping + GenericRecord requiredAvroRecord = buildAvroRecordBySchema( + insertAvroRecord.get(), + requiredSchema, + requiredPos, + recordBuilder); + this.currentRecord = (RowData) avroToRowDataConverter.convert(requiredAvroRecord); + return false; } } - return true; } + return true; } @Override diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java index 46d727262c7cd..e8927dc7f34f3 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java @@ -22,9 +22,11 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.table.HoodieFlinkTable; import org.apache.avro.Schema; import org.apache.flink.configuration.Configuration; @@ -41,13 +43,6 @@ public class CompactionUtil { private static final Logger LOG = LoggerFactory.getLogger(CompactionUtil.class); - /** - * Creates the metaClient. - */ - public static HoodieTableMetaClient createMetaClient(Configuration conf) { - return HoodieTableMetaClient.builder().setBasePath(conf.getString(FlinkOptions.PATH)).setConf(FlinkClientUtil.getHadoopConf()).build(); - } - /** * Gets compaction Instant time. */ @@ -70,7 +65,7 @@ public static String getCompactionInstantTime(HoodieTableMetaClient metaClient) * Sets up the avro schema string into the give configuration {@code conf} * through reading from the hoodie table metadata. * - * @param conf The configuration + * @param conf The configuration */ public static void setAvroSchema(Configuration conf, HoodieTableMetaClient metaClient) throws Exception { TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); @@ -96,4 +91,19 @@ public static void cleanInstant(HoodieTableMetaClient metaClient, HoodieInstant throw new HoodieIOException("Could not remove requested commit " + commitFilePath, e); } } + + public static void rollbackCompaction(HoodieFlinkTable table, Configuration conf) { + String curInstantTime = HoodieActiveTimeline.createNewInstantTime(); + int deltaSeconds = conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS); + HoodieTimeline inflightCompactionTimeline = table.getActiveTimeline() + .filterPendingCompactionTimeline() + .filter(instant -> + instant.getState() == HoodieInstant.State.INFLIGHT + && StreamerUtil.instantTimeDiff(curInstantTime, instant.getTimestamp()) >= deltaSeconds); + inflightCompactionTimeline.getInstants().forEach(inflightInstant -> { + LOG.info("Rollback the pending compaction instant: " + inflightInstant); + table.rollback(table.getContext(), HoodieActiveTimeline.createNewInstantTime(), inflightInstant, true); + table.getMetaClient().reloadActiveTimeline(); + }); + } } diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index fcbdb21b5ec2d..d73b300b233c6 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -26,12 +26,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieRecordLocation; -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.common.util.ReflectionUtils; -import org.apache.hudi.common.util.TablePathUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieMemoryConfig; import org.apache.hudi.config.HoodieStorageConfig; @@ -39,13 +34,11 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.schema.FilebasedSchemaProvider; import org.apache.hudi.streamer.FlinkStreamerConfig; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; @@ -61,7 +54,6 @@ import java.io.StringReader; import java.util.List; import java.util.Locale; -import java.util.Objects; import java.util.Properties; import static org.apache.hudi.common.table.HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER; @@ -136,23 +128,6 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf() { return FlinkClientUtil.getHadoopConf(); } - /** - * Create a payload class via reflection, passing in an ordering/precombine value. - */ - public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal) - throws IOException { - try { - return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass, - new Class[] {GenericRecord.class, Comparable.class}, record, orderingVal); - } catch (Throwable e) { - throw new IOException("Could not create payload for class: " + payloadClass, e); - } - } - - public static HoodieWriteConfig getHoodieClientConfig(FlinkStreamerConfig conf) { - return getHoodieClientConfig(FlinkStreamerConfig.toFlinkConfig(conf)); - } - public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) { HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() @@ -179,7 +154,7 @@ public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) { .withMaxMemoryMaxSize( conf.getInteger(FlinkOptions.WRITE_MERGE_MAX_MEMORY) * 1024 * 1024L, conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024L - ).build()) + ).build()) .forTable(conf.getString(FlinkOptions.TABLE_NAME)) .withStorageConfig(HoodieStorageConfig.newBuilder() .logFileDataBlockMaxSize(conf.getInteger(FlinkOptions.WRITE_LOG_BLOCK_SIZE) * 1024 * 1024) @@ -246,34 +221,16 @@ public static void initTableIfNotExists(Configuration conf) throws IOException { // some of the filesystems release the handles in #close method. } - /** Generates the bucket ID using format {partition path}_{fileID}. */ + /** + * Generates the bucket ID using format {partition path}_{fileID}. + */ public static String generateBucketKey(String partitionPath, String fileId) { return String.format("%s_%s", partitionPath, fileId); } - /** Returns whether the location represents an insert. */ - public static boolean isInsert(HoodieRecordLocation loc) { - return Objects.equals(loc.getInstantTime(), "I"); - } - - public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throws IOException { - LOG.info("Getting table path.."); - for (Path path : userProvidedPaths) { - try { - Option tablePath = TablePathUtils.getTablePath(fs, path); - if (tablePath.isPresent()) { - return tablePath.get().toString(); - } - } catch (HoodieException he) { - LOG.warn("Error trying to get table path from " + path.toString(), he); - } - } - - throw new TableNotFoundException("Unable to find a hudi table for the user provided paths."); - } - /** * Returns whether needs to schedule the async compaction. + * * @param conf The flink configuration. */ public static boolean needsAsyncCompaction(Configuration conf) { @@ -283,6 +240,13 @@ public static boolean needsAsyncCompaction(Configuration conf) { && conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED); } + /** + * Creates the meta client. + */ + public static HoodieTableMetaClient createMetaClient(Configuration conf) { + return HoodieTableMetaClient.builder().setBasePath(conf.getString(FlinkOptions.PATH)).setConf(FlinkClientUtil.getHadoopConf()).build(); + } + /** * Creates the Flink write client. */ @@ -295,14 +259,6 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf, Runti return new HoodieFlinkWriteClient<>(context, getHoodieClientConfig(conf)); } - /** - * Plus the old instant time with given milliseconds and returns. - */ - public static String instantTimePlus(String oldInstant, long milliseconds) { - long oldTime = Long.parseLong(oldInstant); - return String.valueOf(oldTime + milliseconds); - } - /** * Return the median instant time between the given two instant time. */ @@ -312,4 +268,11 @@ public static String medianInstantTime(String highVal, String lowVal) { long median = low + (high - low) / 2; return String.valueOf(median); } + + /** + * Returns the time interval in seconds between the given instant time. + */ + public static long instantTimeDiff(String newInstantTime, String oldInstantTime) { + return Long.parseLong(newInstantTime) - Long.parseLong(oldInstantTime); + } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java b/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java index 8e67159a4a5ad..be4c05219c776 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java @@ -200,7 +200,7 @@ public void testHoodieFlinkCompactor() throws Exception { conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ"); // create metaClient - HoodieTableMetaClient metaClient = CompactionUtil.createMetaClient(conf); + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); // set the table name conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index a2fdf227f4c20..612bb79663306 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -24,7 +24,7 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.configuration.FlinkOptions; -import org.apache.hudi.sink.event.BatchWriteSuccessEvent; +import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.utils.MockCoordinatorExecutor; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; @@ -70,6 +70,23 @@ public void before() throws Exception { TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), context); coordinator.start(); coordinator.setExecutor(new MockCoordinatorExecutor(context)); + + final WriteMetadataEvent event0 = WriteMetadataEvent.builder() + .taskID(0) + .instantTime("") + .writeStatus(Collections.emptyList()) + .isBootstrap(true) + .build(); + + final WriteMetadataEvent event1 = WriteMetadataEvent.builder() + .taskID(1) + .instantTime("") + .writeStatus(Collections.emptyList()) + .isBootstrap(true) + .build(); + + coordinator.handleEventFromOperator(0, event0); + coordinator.handleEventFromOperator(1, event1); } @AfterEach @@ -85,7 +102,7 @@ void testInstantState() { WriteStatus writeStatus = new WriteStatus(true, 0.1D); writeStatus.setPartitionPath("par1"); writeStatus.setStat(new HoodieWriteStat()); - OperatorEvent event0 = BatchWriteSuccessEvent.builder() + OperatorEvent event0 = WriteMetadataEvent.builder() .taskID(0) .instantTime(instant) .writeStatus(Collections.singletonList(writeStatus)) @@ -95,7 +112,7 @@ void testInstantState() { WriteStatus writeStatus1 = new WriteStatus(false, 0.2D); writeStatus1.setPartitionPath("par2"); writeStatus1.setStat(new HoodieWriteStat()); - OperatorEvent event1 = BatchWriteSuccessEvent.builder() + OperatorEvent event1 = WriteMetadataEvent.builder() .taskID(1) .instantTime(instant) .writeStatus(Collections.singletonList(writeStatus1)) @@ -132,7 +149,7 @@ public void testCheckpointAndRestore() throws Exception { public void testReceiveInvalidEvent() { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(1, future); - OperatorEvent event = BatchWriteSuccessEvent.builder() + OperatorEvent event = WriteMetadataEvent.builder() .taskID(0) .instantTime("abc") .writeStatus(Collections.emptyList()) @@ -147,7 +164,7 @@ public void testCheckpointCompleteWithPartialEvents() { final CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(1, future); String instant = coordinator.getInstant(); - OperatorEvent event = BatchWriteSuccessEvent.builder() + OperatorEvent event = WriteMetadataEvent.builder() .taskID(0) .instantTime(instant) .writeStatus(Collections.emptyList()) @@ -163,7 +180,7 @@ public void testCheckpointCompleteWithPartialEvents() { WriteStatus writeStatus1 = new WriteStatus(false, 0.2D); writeStatus1.setPartitionPath("par2"); writeStatus1.setStat(new HoodieWriteStat()); - OperatorEvent event1 = BatchWriteSuccessEvent.builder() + OperatorEvent event1 = WriteMetadataEvent.builder() .taskID(1) .instantTime(instant) .writeStatus(Collections.singletonList(writeStatus1)) @@ -186,20 +203,30 @@ public void testHiveSyncInvoked() throws Exception { coordinator.start(); coordinator.setExecutor(new MockCoordinatorExecutor(context)); + final WriteMetadataEvent event0 = WriteMetadataEvent.builder() + .taskID(0) + .instantTime("") + .writeStatus(Collections.emptyList()) + .isBootstrap(true) + .build(); + + coordinator.handleEventFromOperator(0, event0); + String instant = coordinator.getInstant(); assertNotEquals("", instant); WriteStatus writeStatus = new WriteStatus(true, 0.1D); writeStatus.setPartitionPath("par1"); writeStatus.setStat(new HoodieWriteStat()); - OperatorEvent event0 = BatchWriteSuccessEvent.builder() + + OperatorEvent event1 = WriteMetadataEvent.builder() .taskID(0) .instantTime(instant) .writeStatus(Collections.singletonList(writeStatus)) .isLastBatch(true) .build(); - coordinator.handleEventFromOperator(0, event0); + coordinator.handleEventFromOperator(0, event1); // never throw for hive synchronization now assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1)); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java index e1cb99e1cd87c..b3338a9083cb9 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java @@ -28,7 +28,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sink.event.BatchWriteSuccessEvent; +import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; @@ -135,8 +135,8 @@ public void testCheckpoint() throws Exception { String instant = funcWrapper.getWriteClient().getLastPendingInstant(getTableType()); final OperatorEvent nextEvent = funcWrapper.getNextEvent(); - MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); - List writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses(); + MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); + List writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses(); assertNotNull(writeStatuses); MatcherAssert.assertThat(writeStatuses.size(), is(4)); // write 4 partition files assertThat(writeStatuses.stream() @@ -162,8 +162,8 @@ public void testCheckpoint() throws Exception { assertNotEquals(instant, instant2); final OperatorEvent nextEvent2 = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent2, instanceOf(BatchWriteSuccessEvent.class)); - List writeStatuses2 = ((BatchWriteSuccessEvent) nextEvent2).getWriteStatuses(); + assertThat("The operator expect to send an event", nextEvent2, instanceOf(WriteMetadataEvent.class)); + List writeStatuses2 = ((WriteMetadataEvent) nextEvent2).getWriteStatuses(); assertNotNull(writeStatuses2); assertThat(writeStatuses2.size(), is(0)); // write empty statuses @@ -191,8 +191,8 @@ public void testCheckpointFails() throws Exception { assertNotNull(instant); final OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); - List writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses(); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); + List writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses(); assertNotNull(writeStatuses); assertThat(writeStatuses.size(), is(0)); // no data write @@ -210,7 +210,9 @@ public void testCheckpointFails() throws Exception { } // this returns early because there is no inflight instant - funcWrapper.checkpointFunction(2); + assertThrows(HoodieException.class, + () -> funcWrapper.checkpointFunction(2), + "Timeout(0ms) while waiting for"); // do not sent the write event and fails the checkpoint, // behaves like the last checkpoint is successful. funcWrapper.checkpointFails(2); @@ -232,7 +234,7 @@ public void testInsert() throws Exception { .getLastPendingInstant(getTableType()); final OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -262,7 +264,7 @@ public void testInsertDuplicates() throws Exception { funcWrapper.checkpointFunction(1); OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -298,7 +300,7 @@ public void testUpsert() throws Exception { funcWrapper.checkpointFunction(1); OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -318,7 +320,7 @@ public void testUpsert() throws Exception { .getLastPendingInstant(getTableType()); nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -343,7 +345,7 @@ public void testUpsertWithDelete() throws Exception { funcWrapper.checkpointFunction(1); OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -363,7 +365,7 @@ public void testUpsertWithDelete() throws Exception { .getLastPendingInstant(getTableType()); nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -408,7 +410,7 @@ public void testInsertWithMiniBatches() throws Exception { final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first final OperatorEvent event2 = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", event2, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, event1); funcWrapper.getCoordinator().handleEventFromOperator(0, event2); @@ -470,7 +472,7 @@ public void testInsertWithDeduplication() throws Exception { final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first final OperatorEvent event2 = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", event2, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, event1); funcWrapper.getCoordinator().handleEventFromOperator(0, event2); @@ -534,7 +536,7 @@ public void testInsertWithSmallBufferSize() throws Exception { for (int i = 0; i < 2; i++) { final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first - assertThat("The operator expect to send an event", event, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, event); } assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -569,7 +571,7 @@ public void testInsertWithSmallBufferSize() throws Exception { checkWrittenData(tempFile, expected, 1); } - Map getMiniBatchExpected() { + protected Map getMiniBatchExpected() { Map expected = new HashMap<>(); // the last 2 lines are merged expected.put("par1", "[" @@ -579,6 +581,10 @@ Map getMiniBatchExpected() { return expected; } + protected Map getExpectedBeforeCheckpointComplete() { + return EXPECTED2; + } + @Test public void testIndexStateBootstrap() throws Exception { // open the function and ingest data @@ -592,7 +598,7 @@ public void testIndexStateBootstrap() throws Exception { funcWrapper.checkpointFunction(1); OperatorEvent nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -634,8 +640,10 @@ public void testIndexStateBootstrap() throws Exception { .getLastPendingInstant(getTableType()); nextEvent = funcWrapper.getNextEvent(); - assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class)); - checkWrittenData(tempFile, EXPECTED2); + assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class)); + + Map expected = getExpectedBeforeCheckpointComplete(); + checkWrittenData(tempFile, expected); funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent); assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event"); @@ -673,7 +681,7 @@ public void testWriteExactlyOnce() throws Exception { for (int i = 0; i < 2; i++) { final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first - assertThat("The operator expect to send an event", event, instanceOf(BatchWriteSuccessEvent.class)); + assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class)); funcWrapper.getCoordinator().handleEventFromOperator(0, event); } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java index 17c13a60bb804..07e23b56edc92 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnRead.java @@ -37,8 +37,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.FileSystem; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; import java.io.File; import java.util.Comparator; @@ -83,13 +81,12 @@ protected void checkWrittenData(File baseFile, Map expected, int TestData.checkWrittenDataMOR(fs, latestInstant, baseFile, expected, partitions, schema); } - @Disabled - @Test - public void testIndexStateBootstrap() { - // Ignore the index bootstrap because we only support parquet load now. + @Override + protected Map getExpectedBeforeCheckpointComplete() { + return EXPECTED1; } - Map getMiniBatchExpected() { + protected Map getMiniBatchExpected() { Map expected = new HashMap<>(); // MOR mode merges the messages with the same key. expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]"); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java index 98d1211809bb5..13a71ecb80245 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteMergeOnReadWithCompact.java @@ -45,7 +45,7 @@ public void testIndexStateBootstrap() { // Ignore the index bootstrap because we only support parquet load now. } - Map getMiniBatchExpected() { + protected Map getMiniBatchExpected() { Map expected = new HashMap<>(); // MOR mode merges the messages with the same key. expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]"); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java b/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java index 84ba9da91309a..e78456b0e1634 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/utils/StreamWriteFunctionWrapper.java @@ -27,7 +27,7 @@ import org.apache.hudi.sink.StreamWriteOperatorCoordinator; import org.apache.hudi.sink.bootstrap.BootstrapFunction; import org.apache.hudi.sink.bootstrap.IndexRecord; -import org.apache.hudi.sink.event.BatchWriteSuccessEvent; +import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.partitioner.BucketAssignFunction; import org.apache.hudi.sink.partitioner.BucketAssignOperator; import org.apache.hudi.sink.transform.RowDataToHoodieFunction; @@ -70,15 +70,25 @@ public class StreamWriteFunctionWrapper { private final StreamWriteOperatorCoordinator coordinator; private final MockFunctionInitializationContext functionInitializationContext; - /** Function that converts row data to HoodieRecord. */ + /** + * Function that converts row data to HoodieRecord. + */ private RowDataToHoodieFunction> toHoodieFunction; - /** Function that load index in state. */ + /** + * Function that load index in state. + */ private BootstrapFunction, HoodieRecord> bootstrapFunction; - /** Function that assigns bucket ID. */ + /** + * Function that assigns bucket ID. + */ private BucketAssignFunction, HoodieRecord> bucketAssignerFunction; - /** BucketAssignOperator context. **/ + /** + * BucketAssignOperator context. + **/ private MockBucketAssignOperatorContext bucketAssignOperatorContext; - /** Stream write function. */ + /** + * Stream write function. + */ private StreamWriteFunction, Object> writeFunction; private CompactFunctionWrapper compactFunctionWrapper; @@ -133,8 +143,12 @@ public void openFunction() throws Exception { writeFunction = new StreamWriteFunction<>(conf); writeFunction.setRuntimeContext(runtimeContext); writeFunction.setOperatorEventGateway(gateway); + writeFunction.initializeState(this.functionInitializationContext); writeFunction.open(conf); + // handle the bootstrap event + coordinator.handleEventFromOperator(0, getNextEvent()); + if (asyncCompaction) { compactFunctionWrapper.openFunction(); } @@ -184,7 +198,7 @@ public void close() { writeFunction.processElement(hoodieRecords[0], null, null); } - public BatchWriteSuccessEvent[] getEventBuffer() { + public WriteMetadataEvent[] getEventBuffer() { return this.coordinator.getEventBuffer(); } @@ -201,7 +215,7 @@ public HoodieFlinkWriteClient getWriteClient() { return this.writeFunction.getWriteClient(); } - public void checkpointFunction(long checkpointId) { + public void checkpointFunction(long checkpointId) throws Exception { // checkpoint the coordinator first this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>()); bucketAssignerFunction.snapshotState(null); diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java index c34fd59094b32..f9ff471431255 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java @@ -45,7 +45,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.util.Collection; @@ -56,6 +58,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.utils.TestData.assertRowsEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -254,11 +257,14 @@ void testStreamReadWithDeletes() throws Exception { } @ParameterizedTest - @EnumSource(value = ExecMode.class) - void testWriteAndRead(ExecMode execMode) { + @MethodSource("configParams") + void testWriteAndRead(ExecMode execMode, boolean hiveStylePartitioning) { TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv; Map options = new HashMap<>(); options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + if (hiveStylePartitioning) { + options.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), "true"); + } String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); tableEnv.executeSql(hoodieTableDDL); String insertInto = "insert into t1 values\n" @@ -576,6 +582,19 @@ private enum ExecMode { BATCH, STREAM } + /** + * Return test params => (execution mode, hive style partitioning). + */ + private static Stream configParams() { + Object[][] data = + new Object[][] { + {ExecMode.BATCH, false}, + {ExecMode.BATCH, true}, + {ExecMode.STREAM, false}, + {ExecMode.STREAM, true}}; + return Stream.of(data).map(Arguments::of); + } + private void execInsertSql(TableEnvironment tEnv, String insert) { TableResult tableResult = tEnv.executeSql(insert); // wait to finish diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java index 0a5055a056fa2..f7adf38e47b20 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputPathHandler.java @@ -39,6 +39,14 @@ * InputPathHandler takes in a set of input paths and incremental tables list. Then, classifies the * input paths to incremental, snapshot paths and non-hoodie paths. This is then accessed later to * mutate the JobConf before processing incremental mode queries and snapshot queries. + * + * Note: We are adding jobConf of a mapreduce or spark job. The properties in the jobConf are two + * type: session properties and table properties from metastore. While session property is common + * for all the tables in a query the table properties are unique per table so there is no need to + * check if it belongs to the table for which the path handler is now instantiated. The jobConf has + * all table properties such as name, last modification time and so on which are unique to a table. + * This class is written in such a way that it can handle multiple tables and properties unique to + * a table but for table level property such check is not required. */ public class InputPathHandler { @@ -63,7 +71,6 @@ public InputPathHandler(Configuration conf, Path[] inputPaths, List incr /** * Takes in the original InputPaths and classifies each of them into incremental, snapshot and * non-hoodie InputPaths. The logic is as follows: - * * 1. Check if an inputPath starts with the same basepath as any of the metadata basepaths we know * 1a. If yes, this belongs to a Hoodie table that we already know about. Simply classify this * as incremental or snapshot - We can get the table name of this inputPath from the diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java index d9983bd65f86b..d5d9d9cf52b35 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java @@ -18,13 +18,14 @@ package org.apache.hudi.hadoop.utils; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.JobContext; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.exception.HoodieIOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -73,6 +74,7 @@ public class HoodieHiveUtils { public static final int MAX_COMMIT_ALL = -1; public static final int DEFAULT_LEVELS_TO_BASEPATH = 3; public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode"); + public static final String GLOBALLY_CONSISTENT_READ_TIMESTAMP = "last_replication_timestamp"; public static boolean stopAtCompaction(JobContext job, String tableName) { String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName); 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 b39ee349ae852..26fbdda291710 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 @@ -442,6 +442,7 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map< } HoodieTimeline timeline = HoodieHiveUtils.getTableTimeline(metaClient.getTableConfig().getTableName(), job, metaClient); + HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient -> FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline)); List filteredBaseFiles = new ArrayList<>(); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestGloballyConsistentTimeStampFilteringInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestGloballyConsistentTimeStampFilteringInputFormat.java new file mode 100644 index 0000000000000..50e4a6ed72b32 --- /dev/null +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestGloballyConsistentTimeStampFilteringInputFormat.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.testutils.InputFormatTestUtil; +import org.apache.hudi.hadoop.utils.HoodieHiveUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestGloballyConsistentTimeStampFilteringInputFormat + extends TestHoodieParquetInputFormat { + + @BeforeEach + public void setUp() { + super.setUp(); + } + + @Test + public void testInputFormatLoad() throws IOException { + super.testInputFormatLoad(); + + // set filtering timestamp to 0 now the timeline wont have any commits. + InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "0"); + + Assertions.assertThrows(HoodieIOException.class, () -> inputFormat.getSplits(jobConf, 10)); + Assertions.assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf)); + } + + @Test + public void testInputFormatUpdates() throws IOException { + super.testInputFormatUpdates(); + + // set the globally replicated timestamp to 199 so only 100 is read and update is ignored. + InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "100"); + + FileStatus[] files = inputFormat.listStatus(jobConf); + assertEquals(10, files.length); + + ensureFilesInCommit("5 files have been updated to commit 200. but should get filtered out ", + files,"200", 0); + ensureFilesInCommit("We should see 10 files from commit 100 ", files, "100", 10); + } + + @Override + public void testIncrementalSimple() throws IOException { + // setting filtering timestamp to zero should not in any way alter the result of the test which + // pulls in zero files due to incremental ts being the actual commit time + jobConf.set(HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP, "0"); + super.testIncrementalSimple(); + } + + @Override + public void testIncrementalWithMultipleCommits() throws IOException { + super.testIncrementalWithMultipleCommits(); + + // set globally replicated timestamp to 400 so commits from 500, 600 does not show up + InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "400"); + InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtils.MAX_COMMIT_ALL); + + FileStatus[] files = inputFormat.listStatus(jobConf); + + assertEquals( + 5, files.length,"Pulling ALL commits from 100, should get us the 3 files from 400 commit, 1 file from 300 " + + "commit and 1 file from 200 commit"); + ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit", + files, "400", 3); + ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit", + files, "300", 1); + ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit", + files, "200", 1); + + List commits = Arrays.asList("100", "200", "300", "400", "500", "600"); + for (int idx = 0; idx < commits.size(); ++idx) { + for (int jdx = 0; jdx < commits.size(); ++jdx) { + InputFormatTestUtil.setupIncremental(jobConf, commits.get(idx), HoodieHiveUtils.MAX_COMMIT_ALL); + InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, commits.get(jdx)); + + files = inputFormat.listStatus(jobConf); + + if (jdx <= idx) { + assertEquals(0, files.length,"all commits should be filtered"); + } else { + // only commits upto the timestamp is allowed + for (FileStatus file : files) { + String commitTs = FSUtils.getCommitTime(file.getPath().getName()); + assertTrue(commits.indexOf(commitTs) <= jdx); + assertTrue(commits.indexOf(commitTs) > idx); + } + } + } + } + } +} 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 c4fed987dd011..c45c6146778b9 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 @@ -65,8 +65,8 @@ public class TestHoodieParquetInputFormat { - private HoodieParquetInputFormat inputFormat; - private JobConf jobConf; + protected HoodieParquetInputFormat inputFormat; + protected JobConf jobConf; private final HoodieFileFormat baseFileFormat = HoodieFileFormat.PARQUET; private final String baseFileExtension = baseFileFormat.getFileExtension(); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java index 3a8b19744f228..0287318e47a26 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java @@ -23,10 +23,12 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.jupiter.api.AfterAll; @@ -169,6 +171,21 @@ public void testInputPathHandler() throws IOException { assertTrue(actualComparesToExpected(actualPaths, nonHoodiePaths)); } + @Test + public void testInputPathHandlerWithGloballyReplicatedTimeStamp() throws IOException { + JobConf jobConf = new JobConf(); + jobConf.set(HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP, "1"); + inputPathHandler = new InputPathHandler(dfs.getConf(), inputPaths.toArray( + new Path[inputPaths.size()]), incrementalTables); + List actualPaths = inputPathHandler.getGroupedIncrementalPaths().values().stream() + .flatMap(List::stream).collect(Collectors.toList()); + assertTrue(actualComparesToExpected(actualPaths, incrementalPaths)); + actualPaths = inputPathHandler.getSnapshotPaths(); + assertTrue(actualComparesToExpected(actualPaths, snapshotPaths)); + actualPaths = inputPathHandler.getNonHoodieInputPaths(); + assertTrue(actualComparesToExpected(actualPaths, nonHoodiePaths)); + } + private boolean actualComparesToExpected(List actualPaths, List expectedPaths) { if (actualPaths.size() != expectedPaths.size()) { return false; diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md index 26db278db03f6..68db715a3dd60 100644 --- a/hudi-integ-test/README.md +++ b/hudi-integ-test/README.md @@ -488,3 +488,33 @@ Spark submit with the flag: --saferSchemaEvolution ``` +## Automated tests for N no of yamls in Local Docker environment + +Hudi provides a script to assist you in testing N no of yamls automatically. Checkout the script under +hudi_root/docker folder. +generate_test_suite.sh + +Example command : // execute the command from within docker folder. +./generate_test_suite.sh --execute_test_suite false --include_medium_test_suite_yaml true --include_long_test_suite_yaml true + +By default, generate_test_suite will run sanity test. In addition it supports 3 more yamls. +medium_test_suite, long_test_suite and clustering_test_suite. Users can add the required yamls via command line as per thier +necessity. + +Also, "--execute_test_suite" false will generate all required files and yamls in a local staging directory if users want to inspect them. +To go ahead and execute the same, you can give "--execute_test_suite true". +staging dir: docker/demo/config/test-suite/staging + +Also, there are other additional configs which users can override depending on their needs. +Some of the options are + +--table_type COPY_ON_WRITE/MERGE_ON_READ // refers to table type. +--medium_num_iterations 20 // refers to total iterations medium test suite should run. +--long_num_iterations 100 // refers to total iterations long test suite should run. +--intermittent_delay_mins 1 // refers to delay between successive runs within a single test suite job. +--cluster_num_itr 30 // refers to total iterations for clustering test suite. +--cluster_delay_mins 2 // refers to delay between successive runs for clustering test suite job. +--cluster_exec_itr_count 15 // refers to the iteration at which clustering needs to be triggered. + + + diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 89faa3bbbf942..06c2ea369b13f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -382,8 +382,6 @@ object DataSourceWriteOptions { // Avro Kafka Source configs val KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class" - - // Schema provider class to be set to be used in custom kakfa deserializer - val SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class" - + // Schema to be used in custom kakfa deserializer + val KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA = "hoodie.deltastreamer.source.kafka.value.deserializer.schema" } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala index 32bd9a4396787..d9f64ba9b804e 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.HoodieROTablePathFilter import org.apache.log4j.LogManager +import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -213,11 +214,27 @@ class DefaultSource extends RelationProvider classOf[HoodieROTablePathFilter], classOf[org.apache.hadoop.fs.PathFilter]) + val specifySchema = if (schema == null) { + // Load the schema from the commit meta data. + // Here we should specify the schema to the latest commit schema since + // the table schema evolution. + val tableSchemaResolver = new TableSchemaResolver(metaClient) + try { + Some(SchemaConverters.toSqlType(tableSchemaResolver.getTableAvroSchema) + .dataType.asInstanceOf[StructType]) + } catch { + case _: Throwable => + None // If there is no commit in the table, we can not get the schema + // with tableSchemaResolver, return None here. + } + } else { + Some(schema) + } // simply return as a regular relation DataSource.apply( sparkSession = sqlContext.sparkSession, paths = extraReadPaths, - userSpecifiedSchema = Option(schema), + userSpecifiedSchema = specifySchema, className = formatClassName, options = optParams) .resolveRelation() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala index a1c38bb708e77..27846e15df096 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hudi import scala.collection.JavaConverters._ import java.net.URI import java.util.Locale - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.SparkAdapterSupport @@ -30,7 +29,7 @@ import org.apache.spark.sql.{Column, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.expressions.{And, Cast, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf @@ -106,6 +105,10 @@ object HoodieSqlUtils extends SparkAdapterSupport { } } + def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = { + attrs.filterNot(attr => isMetaField(attr.name)) + } + /** * Get the table location. * @param tableId diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index ed779902574c5..dbd36bee484b7 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -29,11 +29,11 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, Literal, Na import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, DeleteFromTable, InsertAction, LogicalPlan, MergeIntoTable, Project, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.CreateDataSourceTableCommand +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableChangeColumnCommand, AlterTableRenameCommand, CreateDataSourceTableCommand, TruncateTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} import org.apache.spark.sql.hudi.HoodieSqlUtils import org.apache.spark.sql.hudi.HoodieSqlUtils._ -import org.apache.spark.sql.hudi.command.{CreateHoodieTableAsSelectCommand, CreateHoodieTableCommand, DeleteHoodieTableCommand, InsertIntoHoodieTableCommand, MergeIntoHoodieTableCommand, UpdateHoodieTableCommand} +import org.apache.spark.sql.hudi.command.{AlterHoodieTableAddColumnsCommand, AlterHoodieTableChangeColumnCommand, AlterHoodieTableRenameCommand, CreateHoodieTableAsSelectCommand, CreateHoodieTableCommand, DeleteHoodieTableCommand, InsertIntoHoodieTableCommand, MergeIntoHoodieTableCommand, TruncateHoodieTableCommand, UpdateHoodieTableCommand} import org.apache.spark.sql.types.StringType object HoodieAnalysis { @@ -86,6 +86,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan] case CreateTable(table, mode, Some(query)) if query.resolved && isHoodieTable(table) => CreateHoodieTableAsSelectCommand(table, mode, query) + case _=> plan } } @@ -307,6 +308,22 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic case CreateDataSourceTableCommand(table, ignoreIfExists) if isHoodieTable(table) => CreateHoodieTableCommand(table, ignoreIfExists) + // Rewrite the AlterTableAddColumnsCommand to AlterHoodieTableAddColumnsCommand + case AlterTableAddColumnsCommand(tableId, colsToAdd) + if isHoodieTable(tableId, sparkSession) => + AlterHoodieTableAddColumnsCommand(tableId, colsToAdd) + // Rewrite the AlterTableRenameCommand to AlterHoodieTableRenameCommand + case AlterTableRenameCommand(oldName, newName, isView) + if !isView && isHoodieTable(oldName, sparkSession) => + new AlterHoodieTableRenameCommand(oldName, newName, isView) + // Rewrite the AlterTableChangeColumnCommand to AlterHoodieTableChangeColumnCommand + case AlterTableChangeColumnCommand(tableName, columnName, newColumn) + if isHoodieTable(tableName, sparkSession) => + AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn) + // Rewrite TruncateTableCommand to TruncateHoodieTableCommand + case TruncateTableCommand(tableName, partitionSpec) + if isHoodieTable(tableName, sparkSession) => + new TruncateHoodieTableCommand(tableName, partitionSpec) case _ => plan } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala new file mode 100644 index 0000000000000..b513034bda274 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command + +import java.nio.charset.StandardCharsets + +import org.apache.avro.Schema +import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieInstant.State +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.util.{CommitUtils, Option} +import org.apache.hudi.table.HoodieSparkTable + +import scala.collection.JavaConverters._ +import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, HoodieWriterUtils} +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} +import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.util.SchemaUtils + +import scala.util.control.NonFatal + +/** + * Command for add new columns to the hudi table. + */ +case class AlterHoodieTableAddColumnsCommand( + tableId: TableIdentifier, + colsToAdd: Seq[StructField]) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + if (colsToAdd.nonEmpty) { + val table = sparkSession.sessionState.catalog.getTableMetadata(tableId) + // Get the new schema + val newSqlSchema = StructType(table.schema.fields ++ colsToAdd) + val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableId.table) + val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace) + + // Commit with new schema to change the table schema + AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, table, sparkSession) + + // Refresh the new schema to meta + refreshSchemaInMeta(sparkSession, table, newSqlSchema) + } + Seq.empty[Row] + } + + private def refreshSchemaInMeta(sparkSession: SparkSession, table: CatalogTable, + newSqlSchema: StructType): Unit = { + try { + sparkSession.catalog.uncacheTable(tableId.quotedString) + } catch { + case NonFatal(e) => + log.warn(s"Exception when attempting to uncache table ${tableId.quotedString}", e) + } + sparkSession.catalog.refreshTable(table.identifier.unquotedString) + + SchemaUtils.checkColumnNameDuplication( + newSqlSchema.map(_.name), + "in the table definition of " + table.identifier, + conf.caseSensitiveAnalysis) + DDLUtils.checkDataColNames(table, colsToAdd.map(_.name)) + + sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlSchema) + } +} + +object AlterHoodieTableAddColumnsCommand { + /** + * Generate an empty commit with new schema to change the table's schema. + * @param schema The new schema to commit. + * @param table The hoodie table. + * @param sparkSession The spark session. + */ + def commitWithSchema(schema: Schema, table: CatalogTable, sparkSession: SparkSession): Unit = { + val path = getTableLocation(table, sparkSession) + .getOrElse(s"missing location for ${table.identifier}") + + val jsc = new JavaSparkContext(sparkSession.sparkContext) + val client = DataSourceUtils.createHoodieClient(jsc, schema.toString, + path, table.identifier.table, HoodieWriterUtils.parametersWithWriteDefaults(table.storage.properties).asJava) + + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val metaClient = HoodieTableMetaClient.builder().setBasePath(path).setConf(hadoopConf).build() + + val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, metaClient.getTableType) + val instantTime = HoodieActiveTimeline.createNewInstantTime + client.startCommitWithTime(instantTime, commitActionType) + + val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext) + val timeLine = hoodieTable.getActiveTimeline + val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime) + val metadata = new HoodieCommitMetadata + metadata.setOperationType(WriteOperationType.INSERT) + timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8))) + + client.commit(instantTime, jsc.emptyRDD) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala new file mode 100644 index 0000000000000..78334fdfa3e44 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableChangeColumnCommand.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command + +import org.apache.avro.Schema +import org.apache.hudi.AvroConversionUtils +import org.apache.hudi.avro.HoodieAvroUtils +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.exception.HoodieException +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation +import org.apache.spark.sql.types.{StructField, StructType} + +import scala.util.control.NonFatal + +/** + * Command for alter hudi table's column type. + */ +case class AlterHoodieTableChangeColumnCommand( + tableName: TableIdentifier, + columnName: String, + newColumn: StructField) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + val table = catalog.getTableMetadata(tableName) + val resolver = sparkSession.sessionState.conf.resolver + + if (!resolver(columnName, newColumn.name)) { + throw new AnalysisException(s"Can not support change column name for hudi table currently.") + } + // Get the new schema + val newSqlSchema = StructType( + table.dataSchema.fields.map { field => + if (resolver(field.name, columnName)) { + newColumn + } else { + field + } + }) + val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName.table) + val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace) + + val path = getTableLocation(table, sparkSession) + .getOrElse(s"missing location for ${table.identifier}") + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val metaClient = HoodieTableMetaClient.builder().setBasePath(path) + .setConf(hadoopConf).build() + // Validate the compatibility between new schema and origin schema. + validateSchema(newSchema, metaClient) + // Commit new schema to change the table schema + AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, table, sparkSession) + + try { + sparkSession.catalog.uncacheTable(tableName.quotedString) + } catch { + case NonFatal(e) => + log.warn(s"Exception when attempting to uncache table ${tableName.quotedString}", e) + } + sparkSession.catalog.refreshTable(tableName.unquotedString) + // Change the schema in the meta + catalog.alterTableDataSchema(tableName, newSqlSchema) + + Seq.empty[Row] + } + + private def validateSchema(newSchema: Schema, metaClient: HoodieTableMetaClient): Unit = { + val schemaUtil = new TableSchemaResolver(metaClient) + val tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchemaWithoutMetadataFields) + if (!TableSchemaResolver.isSchemaCompatible(tableSchema, newSchema)) { + throw new HoodieException("Failed schema compatibility check for newSchema :" + newSchema + + ", origin table schema :" + tableSchema + ", base path :" + metaClient.getBasePath) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala new file mode 100644 index 0000000000000..2afef51a919b1 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableRenameCommand.scala @@ -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.spark.sql.hudi.command + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.AlterTableRenameCommand +import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation + +/** + * Command for alter hudi table's table name. + */ +class AlterHoodieTableRenameCommand( + oldName: TableIdentifier, + newName: TableIdentifier, + isView: Boolean) + extends AlterTableRenameCommand(oldName, newName, isView) { + + override def run(sparkSession: SparkSession): Seq[Row] = { + if (newName != oldName) { + val catalog = sparkSession.sessionState.catalog + val table = catalog.getTableMetadata(oldName) + val path = getTableLocation(table, sparkSession) + .getOrElse(s"missing location for ${table.identifier}") + + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val metaClient = HoodieTableMetaClient.builder().setBasePath(path) + .setConf(hadoopConf).build() + // Init table with new name. + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(metaClient.getTableConfig.getProperties) + .setTableName(newName.table) + .initTable(hadoopConf, path) + // Call AlterTableRenameCommand#run to rename table in meta. + super.run(sparkSession) + } + Seq.empty[Row] + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala index 04da5229fb1b1..c7fac53416b1c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala @@ -32,7 +32,7 @@ import org.apache.hudi.{HoodieSparkSqlWriter, HoodieWriterUtils} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils} @@ -86,11 +86,16 @@ object InsertIntoHoodieTableCommand { SaveMode.Append } val parameters = HoodieWriterUtils.parametersWithWriteDefaults(config) - val queryData = Dataset.ofRows(sparkSession, query) val conf = sparkSession.sessionState.conf - val alignedQuery = alignOutputFields(queryData, table, insertPartitions, conf) + val alignedQuery = alignOutputFields(query, table, insertPartitions, conf) + // If we create dataframe using the Dataset.ofRows(sparkSession, alignedQuery), + // The nullable attribute of fields will lost. + // In order to pass the nullable attribute to the inputDF, we specify the schema + // of the rdd. + val inputDF = sparkSession.createDataFrame( + Dataset.ofRows(sparkSession, alignedQuery).rdd, alignedQuery.schema) val success = - HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, parameters, alignedQuery)._1 + HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, parameters, inputDF)._1 if (success) { if (refreshTable) { sparkSession.catalog.refreshTable(table.identifier.unquotedString) @@ -110,10 +115,10 @@ object InsertIntoHoodieTableCommand { * @return */ private def alignOutputFields( - query: DataFrame, + query: LogicalPlan, table: CatalogTable, insertPartitions: Map[String, Option[String]], - conf: SQLConf): DataFrame = { + conf: SQLConf): LogicalPlan = { val targetPartitionSchema = table.partitionSchema @@ -123,18 +128,23 @@ object InsertIntoHoodieTableCommand { s"Required partition columns is: ${targetPartitionSchema.json}, Current static partitions " + s"is: ${staticPartitionValues.mkString("," + "")}") + assert(staticPartitionValues.size + query.output.size == table.schema.size, + s"Required select columns count: ${removeMetaFields(table.schema).size}, " + + s"Current select columns(including static partition column) count: " + + s"${staticPartitionValues.size + removeMetaFields(query.output).size},columns: " + + s"(${(removeMetaFields(query.output).map(_.name) ++ staticPartitionValues.keys).mkString(",")})") val queryDataFields = if (staticPartitionValues.isEmpty) { // insert dynamic partition - query.logicalPlan.output.dropRight(targetPartitionSchema.fields.length) + query.output.dropRight(targetPartitionSchema.fields.length) } else { // insert static partition - query.logicalPlan.output + query.output } val targetDataSchema = table.dataSchema // Align for the data fields of the query val dataProjects = queryDataFields.zip(targetDataSchema.fields).map { case (dataAttr, targetField) => - val castAttr = castIfNeeded(dataAttr, + val castAttr = castIfNeeded(dataAttr.withNullability(targetField.nullable), targetField.dataType, conf) - new Column(Alias(castAttr, targetField.name)()) + Alias(castAttr, targetField.name)() } val partitionProjects = if (staticPartitionValues.isEmpty) { // insert dynamic partitions @@ -142,23 +152,23 @@ object InsertIntoHoodieTableCommand { // So we init the partitionAttrPosition with the data schema size. var partitionAttrPosition = targetDataSchema.size targetPartitionSchema.fields.map(f => { - val partitionAttr = query.logicalPlan.output(partitionAttrPosition) + val partitionAttr = query.output(partitionAttrPosition) partitionAttrPosition = partitionAttrPosition + 1 - val castAttr = castIfNeeded(partitionAttr, f.dataType, conf) - new Column(Alias(castAttr, f.name)()) + val castAttr = castIfNeeded(partitionAttr.withNullability(f.nullable), f.dataType, conf) + Alias(castAttr, f.name)() }) } else { // insert static partitions targetPartitionSchema.fields.map(f => { val staticPartitionValue = staticPartitionValues.getOrElse(f.name, s"Missing static partition value for: ${f.name}") - val castAttr = Literal.create(staticPartitionValue, f.dataType) - new Column(Alias(castAttr, f.name)()) + val castAttr = castIfNeeded(Literal.create(staticPartitionValue), f.dataType, conf) + Alias(castAttr, f.name)() }) } // Remove the hoodie meta fileds from the projects as we do not need these to write - val withoutMetaFieldDataProjects = dataProjects.filter(c => !HoodieSqlUtils.isMetaField(c.named.name)) + val withoutMetaFieldDataProjects = dataProjects.filter(c => !HoodieSqlUtils.isMetaField(c.name)) val alignedProjects = withoutMetaFieldDataProjects ++ partitionProjects - query.select(alignedProjects: _*) + Project(alignedProjects, query) } /** diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala new file mode 100644 index 0000000000000..2cce88e799901 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/TruncateHoodieTableCommand.scala @@ -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.spark.sql.hudi.command + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.execution.command.TruncateTableCommand +import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation + +/** + * Command for truncate hudi table. + */ +class TruncateHoodieTableCommand( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec]) + extends TruncateTableCommand(tableName, partitionSpec) { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val table = sparkSession.sessionState.catalog.getTableMetadata(tableName) + val path = getTableLocation(table, sparkSession) + .getOrElse(s"missing location for ${table.identifier}") + val hadoopConf = sparkSession.sessionState.newHadoopConf() + // If we have not specified the partition, truncate will delete all the + // data in the table path include the hoodi.properties. In this case we + // should reInit the table. + val needReInitTable = partitionSpec.isEmpty + + val tableProperties = if (needReInitTable) { + // Create MetaClient + val metaClient = HoodieTableMetaClient.builder().setBasePath(path) + .setConf(hadoopConf).build() + Some(metaClient.getTableConfig.getProperties) + } else { + None + } + // Delete all data in the table directory + super.run(sparkSession) + + if (tableProperties.isDefined) { + // ReInit hoodie.properties + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(tableProperties.get) + .initTable(hadoopConf, path) + } + Seq.empty[Row] + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql index 35dde250aa5d4..280fde59ff30a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/sql-statements.sql @@ -230,6 +230,20 @@ select id, name, price, ts, dt from h1_p order by id; | 6 _insert 10.0 1000 2021-05-08 | +--------------------------------+ +# ALTER TABLE +alter table h1_p rename to h2_p; ++----------+ +| ok | ++----------+ +alter table h2_p add columns(ext0 int); ++----------+ +| ok | ++----------+ +alter table h2_p change column ext0 ext0 bigint; ++----------+ +| ok | ++----------+ + # DROP TABLE drop table h0; +----------+ @@ -246,7 +260,7 @@ drop table h1; | ok | +----------+ -drop table h1_p; +drop table h2_p; +----------+ | ok | +----------+ diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala new file mode 100644 index 0000000000000..ee73823b0c799 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTable.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.types.{LongType, StructField, StructType} + +class TestAlterTable extends TestHoodieSqlBase { + + test("Test Alter Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val tablePath = s"${tmp.getCanonicalPath}/$tableName" + // Create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '$tablePath' + | options ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // Alter table name. + val newTableName = s"${tableName}_1" + spark.sql(s"alter table $tableName rename to $newTableName") + assertResult(false)( + spark.sessionState.catalog.tableExists(new TableIdentifier(tableName)) + ) + assertResult(true) ( + spark.sessionState.catalog.tableExists(new TableIdentifier(newTableName)) + ) + val hadoopConf = spark.sessionState.newHadoopConf() + val metaClient = HoodieTableMetaClient.builder().setBasePath(tablePath) + .setConf(hadoopConf).build() + assertResult(newTableName) ( + metaClient.getTableConfig.getTableName + ) + spark.sql(s"insert into $newTableName values(1, 'a1', 10, 1000)") + + // Add table column + spark.sql(s"alter table $newTableName add columns(ext0 string)") + val table = spark.sessionState.catalog.getTableMetadata(new TableIdentifier(newTableName)) + assertResult(Seq("id", "name", "price", "ts", "ext0")) { + HoodieSqlUtils.removeMetaFields(table.schema).fields.map(_.name) + } + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(1, "a1", 10.0, 1000, null) + ) + // Alter table column type + spark.sql(s"alter table $newTableName change column id id bigint") + assertResult(StructType(Seq(StructField("id", LongType, nullable = true))))( + spark.sql(s"select id from $newTableName").schema) + + // Insert data to the new table. + spark.sql(s"insert into $newTableName values(2, 'a2', 12, 1000, 'e0')") + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(1, "a1", 10.0, 1000, null), + Seq(2, "a2", 12.0, 1000, "e0") + ) + + // Merge data to the new table. + spark.sql( + s""" + |merge into $newTableName t0 + |using ( + | select 1 as id, 'a1' as name, 12 as price, 1001 as ts, 'e0' as ext0 + |) s0 + |on t0.id = s0.id + |when matched then update set * + |when not matched then insert * + """.stripMargin) + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(1, "a1", 12.0, 1001, "e0"), + Seq(2, "a2", 12.0, 1000, "e0") + ) + + // Update data to the new table. + spark.sql(s"update $newTableName set price = 10, ext0 = null where id = 1") + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(1, "a1", 10.0, 1001, null), + Seq(2, "a2", 12.0, 1000, "e0") + ) + spark.sql(s"update $newTableName set price = 10, ext0 = null where id = 2") + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(1, "a1", 10.0, 1001, null), + Seq(2, "a2", 10.0, 1000, null) + ) + + // Delete data from the new table. + spark.sql(s"delete from $newTableName where id = 1") + checkAnswer(s"select id, name, price, ts, ext0 from $newTableName")( + Seq(2, "a2", 10.0, 1000, null) + ) + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala index 610c71f166566..067e49af46f19 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieSqlBase.scala @@ -78,4 +78,13 @@ class TestHoodieSqlBase extends FunSuite with BeforeAndAfterAll { protected def checkAnswer(sql: String)(expects: Seq[Any]*): Unit = { assertResult(expects.map(row => Row(row: _*)).toArray)(spark.sql(sql).collect()) } + + protected def checkException(sql: String)(errorMsg: String): Unit = { + try { + spark.sql(sql) + } catch { + case e: Throwable => + assertResult(errorMsg)(e.getMessage) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala index 98d095b19f9c2..945ccf5382915 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestInsertTable.scala @@ -220,4 +220,68 @@ class TestInsertTable extends TestHoodieSqlBase { ) } } + + test("Test Different Type of Partition Column") { + withTempDir { tmp => + val typeAndValue = Seq( + ("string", "'1000'"), + ("int", 1000), + ("bigint", 10000), + ("timestamp", "'2021-05-20 00:00:00'"), + ("date", "'2021-05-20'") + ) + typeAndValue.foreach { case (partitionType, partitionValue) => + val tableName = generateTableName + // Create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt $partitionType + |) using hudi + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName' + """.stripMargin) + + spark.sql(s"insert into $tableName partition(dt = $partitionValue) select 1, 'a1', 10") + spark.sql(s"insert into $tableName select 2, 'a2', 10, $partitionValue") + checkAnswer(s"select id, name, price, cast(dt as string) from $tableName order by id")( + Seq(1, "a1", 10, removeQuotes(partitionValue).toString), + Seq(2, "a2", 10, removeQuotes(partitionValue).toString) + ) + } + } + } + + test("Test Insert Exception") { + val tableName = generateTableName + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt string + |) using hudi + | partitioned by (dt) + """.stripMargin) + checkException(s"insert into $tableName partition(dt = '2021-06-20')" + + s" select 1, 'a1', 10, '2021-06-20'") ( + "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" + + " count: 5,columns: (1,a1,10,2021-06-20,dt)" + ) + checkException(s"insert into $tableName select 1, 'a1', 10")( + "assertion failed: Required select columns count: 4, Current select columns(including static partition column)" + + " count: 3,columns: (1,a1,10)" + ) + } + + private def removeQuotes(value: Any): Any = { + value match { + case s: String => s.stripPrefix("'").stripSuffix("'") + case _=> value + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala new file mode 100644 index 0000000000000..6a0f0a4062166 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTruncateTable.scala @@ -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.spark.sql.hudi + +class TestTruncateTable extends TestHoodieSqlBase { + + test("Test Truncate Table") { + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + // Create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + // Insert data + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + // Truncate table + spark.sql(s"truncate table $tableName") + checkAnswer(s"select count(1) from $tableName")(Seq(0)) + + // Insert data to the truncated table. + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000) + ) + } + } +} diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml index c44f785e4a4a3..fd63028951047 100644 --- a/hudi-sync/hudi-hive-sync/pom.xml +++ b/hudi-sync/hudi-hive-sync/pom.xml @@ -30,6 +30,8 @@ ${project.parent.basedir} + + 7.6.0.v20120127 @@ -148,6 +150,14 @@ test + + + org.eclipse.jetty.aggregate + jetty-all + test + ${jetty.version} + + org.junit.jupiter junit-jupiter-api diff --git a/hudi-sync/hudi-hive-sync/run_hive_global_commit_tool.sh b/hudi-sync/hudi-hive-sync/run_hive_global_commit_tool.sh new file mode 100755 index 0000000000000..b7e6cd203d2f5 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/run_hive_global_commit_tool.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# A tool to sync the hudi table to hive from different clusters. Similar to HiveSyncTool but syncs it to more +# than one hive cluster ( currently a local and remote cluster). The common timestamp that was synced is stored as a new table property +# This is most useful when we want to ensure that across different hive clusters we want ensure consistent reads. If that is not a requirement +# then it is better to run HiveSyncTool separately. +# Note: +# The tool tries to be transactional but does not guarantee it. If the sync fails midway in one cluster it will try to roll back the committed +# timestamp from already successful sync on other clusters but that can also fail. +# The tool does not roll back any synced partitions but only the timestamp. + +function error_exit { + echo "$1" >&2 ## Send message to stderr. Exclude >&2 if you don't want it that way. + exit "${2:-1}" ## Return a code specified by $2 or 1 by default. +} + +if [ -z "${HADOOP_HOME}" ]; then + error_exit "Please make sure the environment variable HADOOP_HOME is setup" +fi + +if [ -z "${HIVE_HOME}" ]; then + error_exit "Please make sure the environment variable HIVE_HOME is setup" +fi + +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +#Ensure we pick the right jar even for hive11 builds +HUDI_HIVE_UBER_JAR=`ls -c $DIR/../packaging/hudi-hive-bundle/target/hudi-hive-*.jar | grep -v source | head -1` + +if [ -z "$HADOOP_CONF_DIR" ]; then + echo "setting hadoop conf dir" + HADOOP_CONF_DIR="${HADOOP_HOME}/etc/hadoop" +fi + +## Include only specific packages from HIVE_HOME/lib to avoid version mismatches +HIVE_EXEC=`ls ${HIVE_HOME}/lib/hive-exec-*.jar | tr '\n' ':'` +HIVE_SERVICE=`ls ${HIVE_HOME}/lib/hive-service-*.jar | grep -v rpc | tr '\n' ':'` +HIVE_METASTORE=`ls ${HIVE_HOME}/lib/hive-metastore-*.jar | tr '\n' ':'` +HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*.jar | tr '\n' ':'` +if [ -z "${HIVE_JDBC}" ]; then + HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*.jar | grep -v handler | tr '\n' ':'` +fi +HIVE_JACKSON=`ls ${HIVE_HOME}/lib/jackson-*.jar | tr '\n' ':'` +HIVE_NUCLEUS=`ls ${HIVE_HOME}/lib/datanucleus*.jar | tr '\n' ':'` +HIVE_JARS=$HIVE_METASTORE:$HIVE_SERVICE:$HIVE_EXEC:$HIVE_JDBC:$HIVE_JACKSON:$HIVE_NUCLEUS + +HADOOP_HIVE_JARS=${HIVE_JARS}:${HADOOP_HOME}/share/hadoop/common/*:${HADOOP_HOME}/share/hadoop/mapreduce/*:${HADOOP_HOME}/share/hadoop/hdfs/*:${HADOOP_HOME}/share/hadoop/common/lib/*:${HADOOP_HOME}/share/hadoop/hdfs/lib/* + +if ! [ -z "$HIVE_CONF_DIR" ]; then + error_exit "Don't set HIVE_CONF_DIR; use config xml file" +fi + +echo "Running Command : java -cp $HUDI_HIVE_UBER_JAR:${HADOOP_HIVE_JARS}:${HADOOP_CONF_DIR}:${HIVE_HOME}lib/* org.apache.hudi.hive.replication.HiveSyncGlobalCommitTool $@" +java -cp $HUDI_HIVE_UBER_JAR:${HADOOP_HIVE_JARS}:${HADOOP_CONF_DIR}:${HIVE_HOME}lib/* org.apache.hudi.hive.replication.HiveSyncGlobalCommitTool "$@" 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 e4e796295376e..41c419d063a81 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 @@ -104,6 +104,7 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") public Boolean decodePartition = false; + // enhance the similar function in child class public static HiveSyncConfig copy(HiveSyncConfig cfg) { HiveSyncConfig newConfig = new HiveSyncConfig(); newConfig.basePath = cfg.basePath; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 0dbe97f9b8683..7264c8dffea9d 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -58,10 +58,10 @@ public class HiveSyncTool extends AbstractSyncTool { public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - private final HiveSyncConfig cfg; - private HoodieHiveClient hoodieHiveClient = null; - private String snapshotTableName = null; - private Option roTableName = null; + protected final HiveSyncConfig cfg; + protected HoodieHiveClient hoodieHiveClient = null; + protected String snapshotTableName = null; + protected Option roTableName = null; public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { super(configuration.getAllProperties(), fs); @@ -127,8 +127,8 @@ public void syncHoodieTable() { } } } - - private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, + + protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { LOG.info("Trying to sync hoodie table " + tableName + " with base path " + hoodieHiveClient.getBasePath() + " of type " + hoodieHiveClient.getTableType()); 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 3ae94dd6cae67..9d0214595cf75 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,12 +18,6 @@ package org.apache.hudi.hive; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -32,15 +26,22 @@ import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; @@ -60,6 +61,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; + public class HoodieHiveClient extends AbstractSyncHoodieClient { private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync"; @@ -402,7 +405,19 @@ public void updateHiveSQL(String s) { closeQuietly(null, stmt); } } else { - updateHiveSQLUsingHiveDriver(s); + CommandProcessorResponse response = updateHiveSQLUsingHiveDriver(s); + if (response == null) { + throw new HoodieHiveSyncException("Failed in executing SQL null response" + s); + } + if (response.getResponseCode() != 0) { + LOG.error(String.format("Failure in SQL response %s", response.toString())); + if (response.getException() != null) { + throw new HoodieHiveSyncException( + String.format("Failed in executing SQL %s", s), response.getException()); + } else { + throw new HoodieHiveSyncException(String.format("Failed in executing SQL %s", s)); + } + } } } @@ -476,13 +491,58 @@ public Option getLastCommitTimeSynced(String tableName) { } } + public Option getLastReplicatedTime(String tableName) { + // Get the last replicated time from the TBLproperties + try { + Table database = client.getTable(syncConfig.databaseName, tableName); + return Option.ofNullable(database.getParameters().getOrDefault(GLOBALLY_CONSISTENT_READ_TIMESTAMP, null)); + } catch (NoSuchObjectException e) { + LOG.warn("the said table not found in hms " + syncConfig.databaseName + "." + tableName); + return Option.empty(); + } catch (Exception e) { + throw new HoodieHiveSyncException("Failed to get the last replicated time from the database", e); + } + } + + public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { + if (!activeTimeline.filterCompletedInstants().getInstants() + .anyMatch(i -> i.getTimestamp().equals(timeStamp))) { + throw new HoodieHiveSyncException( + "Not a valid completed timestamp " + timeStamp + " for table " + tableName); + } + try { + Table table = client.getTable(syncConfig.databaseName, tableName); + table.putToParameters(GLOBALLY_CONSISTENT_READ_TIMESTAMP, timeStamp); + client.alter_table(syncConfig.databaseName, tableName, table); + } catch (Exception e) { + throw new HoodieHiveSyncException( + "Failed to update last replicated time to " + timeStamp + " for " + tableName, e); + } + } + + public void deleteLastReplicatedTimeStamp(String tableName) { + try { + Table table = client.getTable(syncConfig.databaseName, tableName); + String timestamp = table.getParameters().remove(GLOBALLY_CONSISTENT_READ_TIMESTAMP); + client.alter_table(syncConfig.databaseName, tableName, table); + if (timestamp != null) { + LOG.info("deleted last replicated timestamp " + timestamp + " for table " + tableName); + } + } catch (NoSuchObjectException e) { + // this is ok the table doesn't even exist. + } catch (Exception e) { + throw new HoodieHiveSyncException( + "Failed to delete last replicated timestamp for " + tableName, e); + } + } + public void close() { try { if (connection != null) { connection.close(); } if (client != null) { - Hive.closeCurrent(); + client.close(); client = null; } } catch (SQLException e) { @@ -506,4 +566,4 @@ public void updateLastCommitTimeSynced(String tableName) { throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e); } } -} \ No newline at end of file +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java new file mode 100644 index 0000000000000..19074c80004b3 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.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.hive.replication; + +import com.beust.jcommander.Parameter; +import org.apache.hudi.hive.HiveSyncConfig; + +public class GlobalHiveSyncConfig extends HiveSyncConfig { + @Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters") + public String globallyReplicatedTimeStamp; + + public static GlobalHiveSyncConfig copy(GlobalHiveSyncConfig cfg) { + GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(); + newConfig.basePath = cfg.basePath; + newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; + newConfig.databaseName = cfg.databaseName; + newConfig.hivePass = cfg.hivePass; + newConfig.hiveUser = cfg.hiveUser; + newConfig.partitionFields = cfg.partitionFields; + newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; + newConfig.jdbcUrl = cfg.jdbcUrl; + newConfig.tableName = cfg.tableName; + newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; + newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; + newConfig.verifyMetadataFileListing = cfg.verifyMetadataFileListing; + newConfig.supportTimestamp = cfg.supportTimestamp; + newConfig.decodePartition = cfg.decodePartition; + newConfig.globallyReplicatedTimeStamp = cfg.globallyReplicatedTimeStamp; + return newConfig; + } + + @Override + public String toString() { + return "GlobalHiveSyncConfig{" + super.toString() + + " globallyReplicatedTimeStamp=" + globallyReplicatedTimeStamp + "}"; + } + +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java new file mode 100644 index 0000000000000..19c23b701fe08 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.hive.replication; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.InvalidTableException; +import org.apache.hudi.hive.HiveSyncTool; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.HashMap; +import java.util.Map; + +public class GlobalHiveSyncTool extends HiveSyncTool { + + private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class); + + public GlobalHiveSyncTool(GlobalHiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { + super(cfg, configuration, fs); + } + + @Override + public void syncHoodieTable() { + switch (hoodieHiveClient.getTableType()) { + case COPY_ON_WRITE: + syncHoodieTable(snapshotTableName, false, false); + break; + case MERGE_ON_READ: + // sync a RO table for MOR + syncHoodieTable(roTableName.get(), false, true); + // sync a RT table for MOR + syncHoodieTable(snapshotTableName, true, false); + break; + default: + LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); + throw new InvalidTableException(hoodieHiveClient.getBasePath()); + } + } + + @Override + protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { + super.syncHoodieTable(tableName, useRealtimeInputFormat, readAsOptimized); + if (((GlobalHiveSyncConfig)cfg).globallyReplicatedTimeStamp != null) { + hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, + ((GlobalHiveSyncConfig) cfg).globallyReplicatedTimeStamp); + } + LOG.info("Sync complete for " + tableName); + } + + public void close() { + hoodieHiveClient.close(); + } + + public Map> getLastReplicatedTimeStampMap() { + Map> timeStampMap = new HashMap<>(); + Option timeStamp = hoodieHiveClient.getLastReplicatedTime(snapshotTableName); + timeStampMap.put(snapshotTableName, timeStamp); + if (HoodieTableType.MERGE_ON_READ.equals(hoodieHiveClient.getTableType())) { + Option roTimeStamp = hoodieHiveClient.getLastReplicatedTime(roTableName.get()); + timeStampMap.put(roTableName.get(), roTimeStamp); + } + return timeStampMap; + } + + public void setLastReplicatedTimeStamp(Map> timeStampMap) { + for (String tableName : timeStampMap.keySet()) { + Option timestamp = timeStampMap.get(tableName); + if (timestamp.isPresent()) { + hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, timestamp.get()); + LOG.info("updated timestamp for " + tableName + " to: " + timestamp.get()); + } else { + hoodieHiveClient.deleteLastReplicatedTimeStamp(tableName); + LOG.info("deleted timestamp for " + tableName); + } + } + } + + public static GlobalHiveSyncTool buildGlobalHiveSyncTool(GlobalHiveSyncConfig cfg, HiveConf hiveConf) { + FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); + hiveConf.addResource(fs.getConf()); + return new GlobalHiveSyncTool(cfg, hiveConf, fs); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommit.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommit.java new file mode 100644 index 0000000000000..ad8d03dba19e1 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommit.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive.replication; + +/** + * A interface to allow syncing the Hudi table to all clusters. + */ +public interface HiveSyncGlobalCommit { + + /** + * + * @return whether the commit succeeded to all the clusters. + */ + boolean commit(); + + /** + * + * @return boolean whether the rollback succeeded to all the clusters. + */ + boolean rollback(); +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java new file mode 100644 index 0000000000000..bce84e9fc198f --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.hive.replication; + +import com.beust.jcommander.Parameter; + +import com.beust.jcommander.Parameters; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Properties; +import org.apache.hudi.common.util.StringUtils; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +// TODO: stop extending HiveSyncConfig and take all the variables needed from config file +@Parameters(commandDescription = "A tool to sync the hudi table to hive from different clusters. Similar to HiveSyncTool but syncs it to more" + + "than one hive cluster ( currently a local and remote cluster). The common timestamp that was synced is stored as a new table property " + + "This is most useful when we want to ensure that across different hive clusters we want ensure consistent reads. If that is not a requirement" + + "then it is better to run HiveSyncTool separately." + + "Note: " + + " The tool tries to be transactional but does not guarantee it. If the sync fails midway in one cluster it will try to roll back the committed " + + " timestamp from already successful sync on other clusters but that can also fail." + + " The tool does not roll back any synced partitions but only the timestamp.") +public class HiveSyncGlobalCommitConfig extends GlobalHiveSyncConfig { + + private static final Logger LOG = LogManager.getLogger(HiveSyncGlobalCommitConfig.class); + + public static String LOCAL_HIVE_SITE_URI = "hivesyncglobal.local_hive_site_uri"; + public static String REMOTE_HIVE_SITE_URI = "hivesyncglobal.remote_hive_site_uri"; + public static String CONFIG_FILE_URI = "hivesyncglobal.config_file_uri"; + public static String REMOTE_BASE_PATH = "hivesyncglobal.remote_base_path"; + public static String LOCAL_BASE_PATH = "hivesyncglobal.local_base_path"; + public static String RETRY_ATTEMPTS = "hivesyncglobal.retry_attempts"; + public static String REMOTE_HIVE_SERVER_JDBC_URLS = "hivesyncglobal.remote_hs2_jdbc_urls"; + public static String LOCAL_HIVE_SERVER_JDBC_URLS = "hivesyncglobal.local_hs2_jdbc_urls"; + + @Parameter(names = { + "--config-xml-file"}, description = "path to the config file in Hive", required = true) + public String configFile; + + public Properties properties = new Properties(); + + private boolean finalize = false; + + public void load() throws IOException { + if (finalize) { + throw new RuntimeException("trying to modify finalized config"); + } + finalize = true; + try (InputStream configStream = new FileInputStream(new File(configFile))) { + properties.loadFromXML(configStream); + } + if (StringUtils.isNullOrEmpty(globallyReplicatedTimeStamp)) { + throw new RuntimeException("globally replicated timestamp not set"); + } + } + + GlobalHiveSyncConfig mkGlobalHiveSyncConfig(boolean forRemote) { + GlobalHiveSyncConfig cfg = GlobalHiveSyncConfig.copy(this); + cfg.basePath = forRemote ? properties.getProperty(REMOTE_BASE_PATH) + : properties.getProperty(LOCAL_BASE_PATH, cfg.basePath); + cfg.jdbcUrl = forRemote ? properties.getProperty(REMOTE_HIVE_SERVER_JDBC_URLS) + : properties.getProperty(LOCAL_HIVE_SERVER_JDBC_URLS, cfg.jdbcUrl); + LOG.info("building hivesync config forRemote: " + forRemote + " " + cfg.jdbcUrl + " " + + cfg.basePath); + return cfg; + } + + @Override + public String toString() { + return "HiveSyncGlobalCommitConfig{ " + "configFile=" + configFile + ", properties=" + + properties + ", " + super.toString() + + " }"; + } + + public void storeToXML(OutputStream configStream) throws IOException { + this.properties.storeToXML(configStream, "hivesync global config"); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java new file mode 100644 index 0000000000000..a194eeb2e96ca --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive.replication; + +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SITE_URI; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SITE_URI; + +import com.beust.jcommander.JCommander; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; + +import org.apache.hudi.hive.HoodieHiveSyncException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +public class HiveSyncGlobalCommitTool implements HiveSyncGlobalCommit, AutoCloseable { + + private static final Logger LOG = LogManager.getLogger(HiveSyncGlobalCommitTool.class); + private final HiveSyncGlobalCommitConfig config; + private List replicationStateSyncList; + + private ReplicationStateSync getReplicatedState(boolean forRemote) { + HiveConf hiveConf = new HiveConf(); + // we probably just need to set the metastore URIs + // TODO: figure out how to integrate this in production + // how to load balance between piper HMS,HS2 + // if we have list of uris, we can do something similar to createHiveConf in reairsync + hiveConf.addResource(new Path(config.properties.getProperty( + forRemote ? REMOTE_HIVE_SITE_URI : LOCAL_HIVE_SITE_URI))); + // TODO: get clusterId as input parameters + ReplicationStateSync state = new ReplicationStateSync(config.mkGlobalHiveSyncConfig(forRemote), + hiveConf, forRemote ? "REMOTESYNC" : "LOCALSYNC"); + return state; + } + + @Override + public boolean commit() { + // TODO: add retry attempts + String name = Thread.currentThread().getName(); + try { + for (ReplicationStateSync stateSync : replicationStateSyncList) { + Thread.currentThread().setName(stateSync.getClusterId()); + LOG.info("starting sync for state " + stateSync); + stateSync.sync(); + LOG.info("synced state " + stateSync); + } + } catch (Exception e) { + Thread.currentThread().setName(name); + LOG.error(String.format("Error while trying to commit replication state %s", e.getMessage()), e); + return false; + } finally { + Thread.currentThread().setName(name); + } + + LOG.info("done syncing to all tables, verifying the timestamps..."); + ReplicationStateSync base = replicationStateSyncList.get(0); + boolean success = true; + LOG.info("expecting all timestamps to be similar to: " + base); + for (int idx = 1; idx < replicationStateSyncList.size(); ++idx) { + ReplicationStateSync other = replicationStateSyncList.get(idx); + if (!base.replicationStateIsInSync(other)) { + LOG.error("the timestamp of other : " + other + " is not matching with base: " + base); + success = false; + } + } + return success; + } + + @Override + public boolean rollback() { + for (ReplicationStateSync stateSync : replicationStateSyncList) { + stateSync.rollback(); + } + return true; + } + + public HiveSyncGlobalCommitTool(HiveSyncGlobalCommitConfig config) { + this.config = config; + this.replicationStateSyncList = new ArrayList<>(2); + this.replicationStateSyncList.add(getReplicatedState(false)); + this.replicationStateSyncList.add(getReplicatedState(true)); + } + + private static HiveSyncGlobalCommitConfig getHiveSyncGlobalCommitConfig(String[] args) + throws IOException { + HiveSyncGlobalCommitConfig cfg = new HiveSyncGlobalCommitConfig(); + JCommander cmd = new JCommander(cfg, null, args); + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + cfg.load(); + return cfg; + } + + @Override + public void close() { + for (ReplicationStateSync stateSync : replicationStateSyncList) { + stateSync.close(); + } + } + + public static void main(String[] args) throws IOException, HoodieHiveSyncException { + final HiveSyncGlobalCommitConfig cfg = getHiveSyncGlobalCommitConfig(args); + try (final HiveSyncGlobalCommitTool globalCommitTool = new HiveSyncGlobalCommitTool(cfg)) { + boolean success = globalCommitTool.commit(); + if (!success) { + if (!globalCommitTool.rollback()) { + throw new RuntimeException("not able to rollback failed commit"); + } + } + } catch (Exception e) { + throw new HoodieHiveSyncException( + "not able to commit replicated timestamp", e); + } + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java new file mode 100644 index 0000000000000..bf806fe4b134b --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive.replication; + +import java.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hudi.common.util.Option; + +public class ReplicationStateSync { + + private GlobalHiveSyncTool globalHiveSyncTool; + private final GlobalHiveSyncConfig globalHiveSyncConfig; + private final HiveConf hiveConf; + private Map> replicatedTimeStampMap; + private Map> oldReplicatedTimeStampMap; + private final String clusterId; + + ReplicationStateSync(GlobalHiveSyncConfig conf, HiveConf hiveConf, String uid) { + this.globalHiveSyncConfig = conf; + this.hiveConf = hiveConf; + initGlobalHiveSyncTool(); + replicatedTimeStampMap = globalHiveSyncTool.getLastReplicatedTimeStampMap(); + clusterId = uid; + } + + private void initGlobalHiveSyncTool() { + globalHiveSyncTool = GlobalHiveSyncTool.buildGlobalHiveSyncTool(globalHiveSyncConfig, hiveConf); + } + + public void sync() throws Exception { + // the cluster maybe down by the time we reach here so we refresh our replication + // state right before we set the oldReplicatedTimeStamp to narrow this window. this is a + // liveliness check right before we start. + replicatedTimeStampMap = globalHiveSyncTool.getLastReplicatedTimeStampMap(); + // it is possible sync fails midway and corrupts the table property therefore we should set + // the oldReplicatedTimeStampMap before the sync start so that we attempt to rollback + // this will help in scenario where sync failed due to some bug in hivesync but in case where + // cluster went down halfway through or before sync in this case rollback may also fail and + // that is ok and we want to be alerted to such scenarios. + oldReplicatedTimeStampMap = replicatedTimeStampMap; + globalHiveSyncTool.syncHoodieTable(); + replicatedTimeStampMap = globalHiveSyncTool.getLastReplicatedTimeStampMap(); + } + + public boolean rollback() { + if (oldReplicatedTimeStampMap != null) { + globalHiveSyncTool.setLastReplicatedTimeStamp(oldReplicatedTimeStampMap); + oldReplicatedTimeStampMap = null; + } + return true; + } + + public boolean replicationStateIsInSync(ReplicationStateSync other) { + return globalHiveSyncTool.getLastReplicatedTimeStampMap() + .equals(other.globalHiveSyncTool.getLastReplicatedTimeStampMap()); + } + + @Override + public String toString() { + return "{ clusterId: " + clusterId + " replicatedState: " + replicatedTimeStampMap + " }"; + } + + public String getClusterId() { + return clusterId; + } + + public void close() { + if (globalHiveSyncTool != null) { + globalHiveSyncTool.close(); + globalHiveSyncTool = null; + } + } + +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java new file mode 100644 index 0000000000000..937243393f7f0 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hive; + +import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SITE_URI; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SITE_URI; + +import java.util.Collections; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig; +import org.apache.hudi.hive.replication.HiveSyncGlobalCommitTool; +import org.apache.hudi.hive.testutils.TestCluster; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestHiveSyncGlobalCommitTool { + + @RegisterExtension + public static TestCluster localCluster = new TestCluster(); + @RegisterExtension + public static TestCluster remoteCluster = new TestCluster(); + + private static String DB_NAME = "foo"; + private static String TBL_NAME = "bar"; + + private HiveSyncGlobalCommitConfig getGlobalCommitConfig( + String commitTime, String dbName, String tblName) throws Exception { + HiveSyncGlobalCommitConfig config = new HiveSyncGlobalCommitConfig(); + config.properties.setProperty(LOCAL_HIVE_SITE_URI, localCluster.getHiveSiteXmlLocation()); + config.properties.setProperty(REMOTE_HIVE_SITE_URI, remoteCluster.getHiveSiteXmlLocation()); + config.properties.setProperty(LOCAL_HIVE_SERVER_JDBC_URLS, localCluster.getHiveJdBcUrl()); + config.properties.setProperty(REMOTE_HIVE_SERVER_JDBC_URLS, remoteCluster.getHiveJdBcUrl()); + config.properties.setProperty(LOCAL_BASE_PATH, localCluster.tablePath(dbName, tblName)); + config.properties.setProperty(REMOTE_BASE_PATH, remoteCluster.tablePath(dbName, tblName)); + config.globallyReplicatedTimeStamp = commitTime; + config.hiveUser = System.getProperty("user.name"); + config.hivePass = ""; + config.databaseName = dbName; + config.tableName = tblName; + config.basePath = localCluster.tablePath(dbName, tblName); + config.assumeDatePartitioning = true; + config.usePreApacheInputFormat = false; + config.partitionFields = Collections.singletonList("datestr"); + return config; + } + + private void compareEqualLastReplicatedTimeStamp(HiveSyncGlobalCommitConfig config) throws Exception { + Assertions.assertEquals(localCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), remoteCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), "compare replicated timestamps"); + } + + @BeforeEach + public void setUp() throws Exception { + localCluster.forceCreateDb(DB_NAME); + remoteCluster.forceCreateDb(DB_NAME); + localCluster.dfsCluster.getFileSystem().delete(new Path(localCluster.tablePath(DB_NAME, TBL_NAME)), true); + remoteCluster.dfsCluster.getFileSystem().delete(new Path(remoteCluster.tablePath(DB_NAME, TBL_NAME)), true); + } + + @AfterEach + public void clear() throws Exception { + localCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + remoteCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + } + + @Test + public void testBasicGlobalCommit() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); + Assertions.assertTrue(tool.commit()); + compareEqualLastReplicatedTimeStamp(config); + } + + @Test + public void testBasicRollback() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); + Assertions.assertFalse(localCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + Assertions.assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + // stop the remote cluster hive server to simulate cluster going down + remoteCluster.stopHiveServer2(); + Assertions.assertFalse(tool.commit()); + Assertions.assertEquals(commitTime, localCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + Assertions.assertTrue(tool.rollback()); // do a rollback + Assertions.assertNotEquals(commitTime, localCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + Assertions.assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + remoteCluster.startHiveServer2(); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 4324a64f7b507..09bf7563eb9bf 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -18,26 +18,23 @@ package org.apache.hudi.hive; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.NetworkTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.Option; +import org.apache.hudi.hive.testutils.HiveTestUtil; import org.apache.hudi.hive.util.ConfigUtils; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; -import org.apache.hudi.hive.testutils.HiveTestUtil; -import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Types; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; import org.joda.time.DateTime; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -48,9 +45,11 @@ import java.io.IOException; import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -68,12 +67,12 @@ private static Iterable useJdbcAndSchemaFromCommitMetadata() { } @BeforeEach - public void setUp() throws IOException, InterruptedException { + public void setUp() throws Exception { HiveTestUtil.setUp(); } @AfterEach - public void teardown() throws IOException { + public void teardown() throws Exception { HiveTestUtil.clear(); } @@ -82,117 +81,6 @@ public static void cleanUpClass() { HiveTestUtil.shutdown(); } - /** - * Testing converting array types to Hive field declaration strings. - *

- * Refer to the Parquet-113 spec: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists - */ - @Test - public void testSchemaConvertArray() throws IOException { - // Testing the 3-level annotation structure - MessageType schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() - .optional(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list").named("int_list") - .named("ArrayOfInts"); - - String schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`int_list` ARRAY< int>", schemaString); - - // A array of arrays - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().requiredGroup() - .as(OriginalType.LIST).repeatedGroup().required(PrimitiveType.PrimitiveTypeName.INT32).named("element") - .named("list").named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString); - - // A list of integers - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeated(PrimitiveType.PrimitiveTypeName.INT32) - .named("element").named("int_list").named("ArrayOfInts"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`int_list` ARRAY< int>", schemaString); - - // A list of structs with two fields - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").required(PrimitiveType.PrimitiveTypeName.INT32) - .named("num").named("element").named("tuple_list").named("ArrayOfTuples"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString); - - // A list of structs with a single field - // For this case, since the inner group name is "array", we treat the - // element type as a one-element struct. - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array").named("one_tuple_list") - .named("ArrayOfOneTuples"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); - - // A list of structs with a single field - // For this case, since the inner group name ends with "_tuple", we also treat the - // element type as a one-element struct. - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list_tuple") - .named("one_tuple_list").named("ArrayOfOneTuples2"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); - - // A list of structs with a single field - // Unlike the above two cases, for this the element type is the type of the - // only field in the struct. - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() - .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list").named("one_tuple_list") - .named("ArrayOfOneTuples3"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`one_tuple_list` ARRAY< binary>", schemaString); - - // A list of maps - schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().as(OriginalType.MAP) - .repeatedGroup().as(OriginalType.MAP_KEY_VALUE).required(PrimitiveType.PrimitiveTypeName.BINARY) - .as(OriginalType.UTF8).named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32).named("int_value") - .named("key_value").named("array").named("map_list").named("ArrayOfMaps"); - - schemaString = HiveSchemaUtil.generateSchemaString(schema); - assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString); - } - - @Test - public void testSchemaConvertTimestampMicros() throws IOException { - MessageType schema = Types.buildMessage().optional(PrimitiveType.PrimitiveTypeName.INT64) - .as(OriginalType.TIMESTAMP_MICROS).named("my_element").named("my_timestamp"); - String schemaString = HiveSchemaUtil.generateSchemaString(schema); - // verify backward compatibility - int64 converted to bigint type - assertEquals("`my_element` bigint", schemaString); - // verify new functionality - int64 converted to timestamp type when 'supportTimestamp' is enabled - schemaString = HiveSchemaUtil.generateSchemaString(schema, Collections.emptyList(), true); - assertEquals("`my_element` TIMESTAMP", schemaString); - } - - @Test - public void testSchemaDiffForTimestampMicros() { - MessageType schema = Types.buildMessage().optional(PrimitiveType.PrimitiveTypeName.INT64) - .as(OriginalType.TIMESTAMP_MICROS).named("my_element").named("my_timestamp"); - // verify backward compatibility - int64 converted to bigint type - SchemaDifference schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, - Collections.emptyMap(), Collections.emptyList(), false); - assertEquals("bigint", schemaDifference.getAddColumnTypes().get("`my_element`")); - schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, - schemaDifference.getAddColumnTypes(), Collections.emptyList(), false); - assertTrue(schemaDifference.isEmpty()); - - // verify schema difference is calculated correctly when supportTimestamp is enabled - schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, - Collections.emptyMap(), Collections.emptyList(), true); - assertEquals("TIMESTAMP", schemaDifference.getAddColumnTypes().get("`my_element`")); - schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, - schemaDifference.getAddColumnTypes(), Collections.emptyList(), true); - assertTrue(schemaDifference.isEmpty()); - } - @ParameterizedTest @MethodSource({"useJdbcAndSchemaFromCommitMetadata"}) public void testBasicSync(boolean useJdbc, boolean useSchemaFromCommitMetadata) throws Exception { @@ -246,6 +134,7 @@ public void testBasicSync(boolean useJdbc, boolean useSchemaFromCommitMetadata) hiveClient = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.hiveSyncConfig.tableName); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty()); + //writtenPartitionsSince.add(newPartition.get(0)); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.UPDATE, partitionEvents.iterator().next().eventType, @@ -769,6 +658,136 @@ public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxE "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); } + private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyCommitTime) throws Exception { + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), + hiveClient.getDataSchema().getColumns().size() + 1, + "Hive Schema should match the table schema + partition field"); + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.hiveSyncConfig.tableName).size(),"Table partitions should match the number of partitions we wrote"); + assertEquals(emptyCommitTime, + hiveClient.getLastCommitTimeSynced(HiveTestUtil.hiveSyncConfig.tableName).get(),"The last commit that was sycned should be updated in the TBLPROPERTIES"); + + // make sure correct schema is picked + Schema schema = SchemaTestUtil.getSimpleSchema(); + for (Field field : schema.getFields()) { + assertEquals(field.schema().getType().getName(), + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get(field.name()).toLowerCase(), + String.format("Hive Schema Field %s was added", field)); + } + assertEquals("string", + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); + assertEquals(schema.getFields().size() + 1 + HoodieRecord.HOODIE_META_COLUMNS.size(), + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(),"Hive Schema fields size"); + } + + @ParameterizedTest + @MethodSource("useJdbc") + public void testPickingOlderParquetFileIfLatestIsEmptyCommit(boolean useJdbc) throws Exception { + HiveTestUtil.hiveSyncConfig.useJdbc = useJdbc; + final String commitTime = "100"; + HiveTestUtil.createCOWTable(commitTime, 1, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + // create empty commit + final String emptyCommitTime = "200"; + HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + verifyOldParquetFileTest(hiveClient, emptyCommitTime); + } + + @ParameterizedTest + @MethodSource("useJdbc") + public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(boolean useJdbc) throws Exception { + HiveTestUtil.hiveSyncConfig.useJdbc = useJdbc; + final String commitTime = "100"; + HiveTestUtil.createCOWTable(commitTime, 1, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + + // evolve the schema + DateTime dateTime = DateTime.now().plusDays(6); + String commitTime2 = "101"; + HiveTestUtil.addCOWPartitions(1, false, true, dateTime, commitTime2); + + // create empty commit + final String emptyCommitTime = "200"; + HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime); + + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertFalse( + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + + // now delete the evolved commit instant + Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + hiveClient.getActiveTimeline().getInstants() + .filter(inst -> inst.getTimestamp().equals(commitTime2)) + .findFirst().get().getFileName()); + assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); + + try { + tool.syncHoodieTable(); + } catch (RuntimeException e) { + // we expect the table sync to fail + } + + // table should not be synced yet + assertFalse( + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist at all"); + } + + @ParameterizedTest + @MethodSource("useJdbc") + public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTable(boolean useJdbc) throws Exception { + HiveTestUtil.hiveSyncConfig.useJdbc = useJdbc; + final String commitTime = "100"; + HiveTestUtil.createCOWTable(commitTime, 1, true); + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + // create empty commit + final String emptyCommitTime = "200"; + HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); + //HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertFalse( + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + verifyOldParquetFileTest(hiveClient, emptyCommitTime); + + // evolve the schema + DateTime dateTime = DateTime.now().plusDays(6); + String commitTime2 = "301"; + HiveTestUtil.addCOWPartitions(1, false, true, dateTime, commitTime2); + //HiveTestUtil.createCommitFileWithSchema(commitMetadata, "400", false); // create another empty commit + //HiveTestUtil.createCommitFile(commitMetadata, "400"); // create another empty commit + + tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + HoodieHiveClient hiveClientLatest = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + // now delete the evolved commit instant + Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + hiveClientLatest.getActiveTimeline().getInstants() + .filter(inst -> inst.getTimestamp().equals(commitTime2)) + .findFirst().get().getFileName()); + assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); + try { + tool.syncHoodieTable(); + } catch (RuntimeException e) { + // we expect the table sync to fail + } + + // old sync values should be left intact + verifyOldParquetFileTest(hiveClient, emptyCommitTime); + } + @ParameterizedTest @MethodSource("useJdbc") public void testTypeConverter(boolean useJdbc) throws Exception { @@ -807,5 +826,4 @@ public void testTypeConverter(boolean useJdbc) throws Exception { .containsValue("BIGINT"), errorMsg); hiveClient.updateHiveSQL(dropTableSql); } - } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index ac083ab0776e5..66343bfd19de1 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.metastore.HiveMetaStore; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.IHMSHandler; @@ -78,6 +79,7 @@ public class HiveTestService { private ExecutorService executorService; private TServer tServer; private HiveServer2 hiveServer; + private HiveConf serverConf; public HiveTestService(Configuration hadoopConf) throws IOException { this.workDir = Files.createTempDirectory(System.currentTimeMillis() + "-").toFile().getAbsolutePath(); @@ -88,6 +90,14 @@ public Configuration getHadoopConf() { return hadoopConf; } + public TServer getHiveMetaStore() { + return tServer; + } + + public HiveConf getServerConf() { + return serverConf; + } + public HiveServer2 start() throws IOException { Objects.requireNonNull(workDir, "The work dir must be set before starting cluster."); @@ -102,10 +112,10 @@ public HiveServer2 start() throws IOException { FileIOUtils.deleteDirectory(file); } - HiveConf serverConf = configureHive(hadoopConf, localHiveLocation); + serverConf = configureHive(hadoopConf, localHiveLocation); executorService = Executors.newSingleThreadExecutor(); - tServer = startMetaStore(bindIP, metastorePort, serverConf); + tServer = startMetaStore(bindIP, serverConf); serverConf.set("hive.in.test", "true"); hiveServer = startHiveServer(serverConf); @@ -116,7 +126,7 @@ public HiveServer2 start() throws IOException { } else { serverHostname = bindIP; } - if (!waitForServerUp(serverConf, serverHostname, metastorePort, CONNECTION_TIMEOUT)) { + if (!waitForServerUp(serverConf, serverHostname, CONNECTION_TIMEOUT)) { throw new IOException("Waiting for startup of standalone server"); } @@ -163,9 +173,17 @@ public String getJdbcHive2Url() { public HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException { conf.set("hive.metastore.local", "false"); - conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort); + int port = metastorePort; + if (conf.get(HiveConf.ConfVars.METASTORE_SERVER_PORT.varname, null) == null) { + conf.setInt(ConfVars.METASTORE_SERVER_PORT.varname, metastorePort); + } else { + port = conf.getInt(ConfVars.METASTORE_SERVER_PORT.varname, metastorePort); + } + if (conf.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, null) == null) { + conf.setInt(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort); + } + conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + port); conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP); - conf.setInt(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort); // The following line to turn of SASL has no effect since HiveAuthFactory calls // 'new HiveConf()'. This is fixed by https://issues.apache.org/jira/browse/HIVE-6657, // in Hive 0.14. @@ -191,8 +209,9 @@ public HiveConf configureHive(Configuration conf, String localHiveLocation) thro return new HiveConf(conf, this.getClass()); } - private boolean waitForServerUp(HiveConf serverConf, String hostname, int port, int timeout) { + private boolean waitForServerUp(HiveConf serverConf, String hostname, int timeout) { long start = System.currentTimeMillis(); + int port = serverConf.getIntVar(HiveConf.ConfVars.METASTORE_SERVER_PORT); while (true) { try { new HiveMetaStoreClient(serverConf); @@ -288,11 +307,12 @@ protected TSocket acceptImpl() throws TTransportException { } } - public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throws IOException { + public TServer startMetaStore(String forceBindIP, HiveConf conf) throws IOException { try { // Server will create new threads up to max as necessary. After an idle // period, it will destory threads to keep the number of threads in the // pool to min. + int port = conf.getIntVar(HiveConf.ConfVars.METASTORE_SERVER_PORT); int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS); int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS); boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 1d6bfb4426462..46f95f616a1a6 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -123,10 +123,10 @@ public static void setUp() throws IOException, InterruptedException { public static void clear() throws IOException { fileSystem.delete(new Path(hiveSyncConfig.basePath), true); HoodieTableMetaClient.withPropertyBuilder() - .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) - .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .setTableType(HoodieTableType.COPY_ON_WRITE) + .setTableName(hiveSyncConfig.tableName) + .setPayloadClass(HoodieAvroPayload.class) + .initTable(configuration, hiveSyncConfig.basePath); HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), fileSystem); for (String tableName : createdTablesSet) { @@ -158,10 +158,10 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo Path path = new Path(hiveSyncConfig.basePath); FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient.withPropertyBuilder() - .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) - .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .setTableType(HoodieTableType.COPY_ON_WRITE) + .setTableName(hiveSyncConfig.tableName) + .setPayloadClass(HoodieAvroPayload.class) + .initTable(configuration, hiveSyncConfig.basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); @@ -173,15 +173,15 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo } public static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions, - boolean createDeltaCommit, boolean useSchemaFromCommitMetadata) + boolean createDeltaCommit, boolean useSchemaFromCommitMetadata) throws IOException, URISyntaxException, InterruptedException { Path path = new Path(hiveSyncConfig.basePath); FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient.withPropertyBuilder() - .setTableType(HoodieTableType.MERGE_ON_READ) - .setTableName(hiveSyncConfig.tableName) - .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, hiveSyncConfig.basePath); + .setTableType(HoodieTableType.MERGE_ON_READ) + .setTableName(hiveSyncConfig.tableName) + .setPayloadClass(HoodieAvroPayload.class) + .initTable(configuration, hiveSyncConfig.basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); @@ -189,25 +189,25 @@ public static void createMORTable(String commitTime, String deltaCommitTime, int HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, useSchemaFromCommitMetadata, dateTime, commitTime); createdTablesSet - .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); + .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); createdTablesSet .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); addSchemaToCommitMetadata(compactionMetadata, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY), - useSchemaFromCommitMetadata); + useSchemaFromCommitMetadata); createCompactionCommitFile(compactionMetadata, commitTime); if (createDeltaCommit) { // Write a delta commit HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), true, - useSchemaFromCommitMetadata); + useSchemaFromCommitMetadata); createDeltaCommitFile(deltaMetadata, deltaCommitTime); } } public static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, - boolean useSchemaFromCommitMetadata, DateTime startFrom, String instantTime) throws IOException, URISyntaxException { + boolean useSchemaFromCommitMetadata, DateTime startFrom, String instantTime) throws IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); @@ -215,7 +215,7 @@ public static void addCOWPartitions(int numberOfPartitions, boolean isParquetSch } public static void addCOWPartition(String partitionPath, boolean isParquetSchemaSimple, - boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException { + boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = createPartition(partitionPath, isParquetSchemaSimple, useSchemaFromCommitMetadata, instantTime); createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); @@ -223,7 +223,7 @@ public static void addCOWPartition(String partitionPath, boolean isParquetSchema } public static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean isLogSchemaSimple, - boolean useSchemaFromCommitMetadata, DateTime startFrom, String instantTime, String deltaCommitTime) + boolean useSchemaFromCommitMetadata, DateTime startFrom, String instantTime, String deltaCommitTime) throws IOException, URISyntaxException, InterruptedException { HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime); @@ -233,7 +233,7 @@ public static void addMORPartitions(int numberOfPartitions, boolean isParquetSch commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); addSchemaToCommitMetadata(compactionMetadata, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY), - useSchemaFromCommitMetadata); + useSchemaFromCommitMetadata); createCompactionCommitFile(compactionMetadata, instantTime); HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple, useSchemaFromCommitMetadata); @@ -241,7 +241,7 @@ public static void addMORPartitions(int numberOfPartitions, boolean isParquetSch } private static HoodieCommitMetadata createLogFiles(Map> partitionWriteStats, - boolean isLogSchemaSimple, boolean useSchemaFromCommitMetadata) + boolean isLogSchemaSimple, boolean useSchemaFromCommitMetadata) throws InterruptedException, IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); for (Entry> wEntry : partitionWriteStats.entrySet()) { @@ -261,7 +261,7 @@ private static HoodieCommitMetadata createLogFiles(Map getCreatedTablesSet() { return createdTablesSet; } -} +} \ No newline at end of file diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java new file mode 100644 index 0000000000000..6a077e10a8330 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestCluster.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.hive.testutils; + +import org.apache.hudi.avro.HoodieAvroWriteSupport; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.bloom.BloomFilterFactory; +import org.apache.hudi.common.bloom.BloomFilterTypeCode; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.SchemaTestUtil; +import org.apache.hudi.common.testutils.minicluster.HdfsTestService; +import org.apache.hudi.common.util.FileIOUtils; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hive.service.server.HiveServer2; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.runners.model.InitializationError; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.fail; + +public class TestCluster implements BeforeAllCallback, AfterAllCallback, + BeforeEachCallback, AfterEachCallback { + private HdfsTestService hdfsTestService; + public HiveTestService hiveTestService; + private Configuration conf; + public HiveServer2 server2; + private static volatile int port = 9083; + public MiniDFSCluster dfsCluster; + DateTimeFormatter dtfOut; + public File hiveSiteXml; + private IMetaStoreClient client; + + @Override + public void beforeAll(ExtensionContext context) throws Exception { + setup(); + } + + @Override + public void afterAll(ExtensionContext context) throws Exception { + shutDown(); + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + } + + public void setup() throws Exception { + hdfsTestService = new HdfsTestService(); + dfsCluster = hdfsTestService.start(true); + + conf = hdfsTestService.getHadoopConf(); + conf.setInt(ConfVars.METASTORE_SERVER_PORT.varname, port++); + conf.setInt(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, port++); + conf.setInt(ConfVars.HIVE_SERVER2_WEBUI_PORT.varname, port++); + hiveTestService = new HiveTestService(conf); + server2 = hiveTestService.start(); + dtfOut = DateTimeFormat.forPattern("yyyy/MM/dd"); + hiveSiteXml = File.createTempFile("hive-site", ".xml"); + hiveSiteXml.deleteOnExit(); + try (OutputStream os = new FileOutputStream(hiveSiteXml)) { + hiveTestService.getServerConf().writeXml(os); + } + client = HiveMetaStoreClient.newSynchronizedClient( + RetryingMetaStoreClient.getProxy(hiveTestService.getServerConf(), true)); + } + + public Configuration getConf() { + return this.conf; + } + + public String getHiveSiteXmlLocation() { + return hiveSiteXml.getAbsolutePath(); + } + + public IMetaStoreClient getHMSClient() { + return client; + } + + public String getHiveJdBcUrl() { + return "jdbc:hive2://127.0.0.1:" + conf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname) + ""; + } + + public String tablePath(String dbName, String tableName) throws Exception { + return dbPath(dbName) + "/" + tableName; + } + + private String dbPath(String dbName) throws Exception { + return dfsCluster.getFileSystem().getWorkingDirectory().toString() + "/" + dbName; + } + + public void forceCreateDb(String dbName) throws Exception { + try { + getHMSClient().dropDatabase(dbName); + } catch (NoSuchObjectException e) { + System.out.println("db does not exist but its ok " + dbName); + } + Database db = new Database(dbName, "", dbPath(dbName), new HashMap<>()); + getHMSClient().createDatabase(db); + } + + public void createCOWTable(String commitTime, int numberOfPartitions, String dbName, String tableName) + throws Exception { + String tablePathStr = tablePath(dbName, tableName); + Path path = new Path(tablePathStr); + FileIOUtils.deleteDirectory(new File(path.toString())); + HoodieTableMetaClient.withPropertyBuilder() + .setTableType(HoodieTableType.COPY_ON_WRITE) + .setTableName(tableName) + .setPayloadClass(HoodieAvroPayload.class) + .initTable(conf, path.toString()); + boolean result = dfsCluster.getFileSystem().mkdirs(path); + if (!result) { + throw new InitializationError("cannot initialize table"); + } + DateTime dateTime = DateTime.now(); + HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime, commitTime, path.toString()); + createCommitFile(commitMetadata, commitTime, path.toString()); + } + + private void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime, String basePath) throws IOException { + byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); + Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + HoodieTimeline.makeCommitFileName(commitTime)); + FSDataOutputStream fsout = dfsCluster.getFileSystem().create(fullPath, true); + fsout.write(bytes); + fsout.close(); + } + + private HoodieCommitMetadata createPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, + DateTime startFrom, String commitTime, String basePath) throws IOException, URISyntaxException { + startFrom = startFrom.withTimeAtStartOfDay(); + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + for (int i = 0; i < numberOfPartitions; i++) { + String partitionPath = dtfOut.print(startFrom); + Path partPath = new Path(basePath + "/" + partitionPath); + dfsCluster.getFileSystem().makeQualified(partPath); + dfsCluster.getFileSystem().mkdirs(partPath); + List writeStats = createTestData(partPath, isParquetSchemaSimple, commitTime); + startFrom = startFrom.minusDays(1); + writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s)); + } + return commitMetadata; + } + + private List createTestData(Path partPath, boolean isParquetSchemaSimple, String commitTime) + throws IOException, URISyntaxException { + List writeStats = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + // Create 5 files + String fileId = UUID.randomUUID().toString(); + Path filePath = new Path(partPath.toString() + "/" + FSUtils + .makeDataFileName(commitTime, "1-0-1", fileId)); + generateParquetData(filePath, isParquetSchemaSimple); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(fileId); + writeStat.setPath(filePath.toString()); + writeStats.add(writeStat); + } + return writeStats; + } + + @SuppressWarnings({"unchecked", "deprecation"}) + private void generateParquetData(Path filePath, boolean isParquetSchemaSimple) + throws IOException, URISyntaxException { + Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema()); + org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema); + BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1, + BloomFilterTypeCode.SIMPLE.name()); + HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter); + ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024, + ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, + ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, dfsCluster.getFileSystem().getConf()); + + List testRecords = (isParquetSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100) + : SchemaTestUtil.generateEvolvedTestRecords(100, 100)); + testRecords.forEach(s -> { + try { + writer.write(s); + } catch (IOException e) { + fail("IOException while writing test records as parquet" + e.toString()); + } + }); + writer.close(); + } + + public HiveConf getHiveConf() { + return server2.getHiveConf(); + } + + public void stopHiveServer2() { + if (server2 != null) { + server2.stop(); + server2 = null; + } + } + + public void startHiveServer2() { + if (server2 == null) { + server2 = new HiveServer2(); + server2.init(hiveTestService.getServerConf()); + server2.start(); + } + } + + public void shutDown() { + stopHiveServer2(); + client.close(); + hiveTestService.getHiveMetaStore().stop(); + hdfsTestService.stop(); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestHiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestHiveSyncGlobalCommitTool.java new file mode 100644 index 0000000000000..980374e0baa4e --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/TestHiveSyncGlobalCommitTool.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS 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.hive.testutils; + +import org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig; +import org.apache.hudi.hive.replication.HiveSyncGlobalCommitTool; + +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SITE_URI; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SITE_URI; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHiveSyncGlobalCommitTool { + + TestCluster localCluster; + TestCluster remoteCluster; + + private static String DB_NAME = "foo"; + private static String TBL_NAME = "bar"; + + private HiveSyncGlobalCommitConfig getGlobalCommitConfig( + String commitTime, String dbName, String tblName) throws Exception { + HiveSyncGlobalCommitConfig config = new HiveSyncGlobalCommitConfig(); + config.properties.setProperty(LOCAL_HIVE_SITE_URI, localCluster.getHiveSiteXmlLocation()); + config.properties.setProperty(REMOTE_HIVE_SITE_URI, remoteCluster.getHiveSiteXmlLocation()); + config.properties.setProperty(LOCAL_HIVE_SERVER_JDBC_URLS, localCluster.getHiveJdBcUrl()); + config.properties.setProperty(REMOTE_HIVE_SERVER_JDBC_URLS, remoteCluster.getHiveJdBcUrl()); + config.properties.setProperty(LOCAL_BASE_PATH, localCluster.tablePath(dbName, tblName)); + config.properties.setProperty(REMOTE_BASE_PATH, remoteCluster.tablePath(dbName, tblName)); + config.globallyReplicatedTimeStamp = commitTime; + config.hiveUser = System.getProperty("user.name"); + config.hivePass = ""; + config.databaseName = dbName; + config.tableName = tblName; + config.basePath = localCluster.tablePath(dbName, tblName); + config.assumeDatePartitioning = true; + config.usePreApacheInputFormat = false; + config.partitionFields = Collections.singletonList("datestr"); + return config; + } + + private void compareEqualLastReplicatedTimeStamp(HiveSyncGlobalCommitConfig config) throws Exception { + assertEquals(localCluster.getHMSClient().getTable(config.databaseName, config.tableName).getParameters().get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), + remoteCluster.getHMSClient().getTable(config.databaseName, config.tableName).getParameters().get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), + "compare replicated timestamps"); + } + + @BeforeEach + public void setUp() throws Exception { + localCluster = new TestCluster(); + localCluster.setup(); + remoteCluster = new TestCluster(); + remoteCluster.setup(); + localCluster.forceCreateDb(DB_NAME); + remoteCluster.forceCreateDb(DB_NAME); + localCluster.dfsCluster.getFileSystem().delete(new Path(localCluster.tablePath(DB_NAME, TBL_NAME)), true); + remoteCluster.dfsCluster.getFileSystem().delete(new Path(remoteCluster.tablePath(DB_NAME, TBL_NAME)), true); + } + + @AfterEach + public void clear() throws Exception { + localCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + remoteCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + localCluster.shutDown(); + remoteCluster.shutDown(); + } + + @Test + public void testBasicGlobalCommit() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); + assertTrue(tool.commit()); + compareEqualLastReplicatedTimeStamp(config); + } + + @Test + public void testBasicRollback() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); + assertFalse(localCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + // stop the remote cluster hive server to simulate cluster going down + remoteCluster.stopHiveServer2(); + assertFalse(tool.commit()); + assertEquals(commitTime, localCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + assertTrue(tool.rollback()); // do a rollback + assertNotEquals(commitTime, localCluster.getHMSClient() + .getTable(config.databaseName, config.tableName).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + remoteCluster.startHiveServer2(); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/TestHiveSchemaUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/TestHiveSchemaUtil.java new file mode 100644 index 0000000000000..9c8ffc106db81 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/util/TestHiveSchemaUtil.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.hive.util; + +import org.apache.hudi.hive.SchemaDifference; + +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Collections; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHiveSchemaUtil { + + /** + * Testing converting array types to Hive field declaration strings. + *

+ * Refer to the Parquet-113 spec: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + */ + @Test + public void testSchemaConvertArray() throws IOException { + // Testing the 3-level annotation structure + MessageType schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() + .optional(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list").named("int_list") + .named("ArrayOfInts"); + + String schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`int_list` ARRAY< int>", schemaString); + + // A array of arrays + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().requiredGroup() + .as(OriginalType.LIST).repeatedGroup().required(PrimitiveType.PrimitiveTypeName.INT32).named("element") + .named("list").named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString); + + // A list of integers + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeated(PrimitiveType.PrimitiveTypeName.INT32) + .named("element").named("int_list").named("ArrayOfInts"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`int_list` ARRAY< int>", schemaString); + + // A list of structs with two fields + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").required(PrimitiveType.PrimitiveTypeName.INT32) + .named("num").named("element").named("tuple_list").named("ArrayOfTuples"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString); + + // A list of structs with a single field + // For this case, since the inner group name is "array", we treat the + // element type as a one-element struct. + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array").named("one_tuple_list") + .named("ArrayOfOneTuples"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); + + // A list of structs with a single field + // For this case, since the inner group name ends with "_tuple", we also treat the + // element type as a one-element struct. + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list_tuple") + .named("one_tuple_list").named("ArrayOfOneTuples2"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString); + + // A list of structs with a single field + // Unlike the above two cases, for this the element type is the type of the + // only field in the struct. + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup() + .required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list").named("one_tuple_list") + .named("ArrayOfOneTuples3"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`one_tuple_list` ARRAY< binary>", schemaString); + + // A list of maps + schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().as(OriginalType.MAP) + .repeatedGroup().as(OriginalType.MAP_KEY_VALUE).required(PrimitiveType.PrimitiveTypeName.BINARY) + .as(OriginalType.UTF8).named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32).named("int_value") + .named("key_value").named("array").named("map_list").named("ArrayOfMaps"); + + schemaString = HiveSchemaUtil.generateSchemaString(schema); + assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString); + } + + @Test + public void testSchemaConvertTimestampMicros() throws IOException { + MessageType schema = Types.buildMessage().optional(PrimitiveType.PrimitiveTypeName.INT64) + .as(OriginalType.TIMESTAMP_MICROS).named("my_element").named("my_timestamp"); + String schemaString = HiveSchemaUtil.generateSchemaString(schema); + // verify backward compatibility - int64 converted to bigint type + assertEquals("`my_element` bigint", schemaString); + // verify new functionality - int64 converted to timestamp type when 'supportTimestamp' is enabled + schemaString = HiveSchemaUtil.generateSchemaString(schema, Collections.emptyList(), true); + assertEquals("`my_element` TIMESTAMP", schemaString); + } + + @Test + public void testSchemaDiffForTimestampMicros() { + MessageType schema = Types.buildMessage().optional(PrimitiveType.PrimitiveTypeName.INT64) + .as(OriginalType.TIMESTAMP_MICROS).named("my_element").named("my_timestamp"); + // verify backward compatibility - int64 converted to bigint type + SchemaDifference schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, + Collections.emptyMap(), Collections.emptyList(), false); + assertEquals("bigint", schemaDifference.getAddColumnTypes().get("`my_element`")); + schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, + schemaDifference.getAddColumnTypes(), Collections.emptyList(), false); + assertTrue(schemaDifference.isEmpty()); + + // verify schema difference is calculated correctly when supportTimestamp is enabled + schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, + Collections.emptyMap(), Collections.emptyList(), true); + assertEquals("TIMESTAMP", schemaDifference.getAddColumnTypes().get("`my_element`")); + schemaDifference = HiveSchemaUtil.getSchemaDifference(schema, + schemaDifference.getAddColumnTypes(), Collections.emptyList(), true); + assertTrue(schemaDifference.isEmpty()); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 65cb4048a39e9..a4dc741272417 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -149,11 +149,11 @@ private String getSchemaFromLatestInstant() throws Exception { private int doCluster(JavaSparkContext jsc) throws Exception { String schemaStr = getSchemaFromLatestInstant(); - SparkRDDWriteClient client = - UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); - JavaRDD writeResponse = - client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses(); - return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + JavaRDD writeResponse = + client.cluster(cfg.clusteringInstantTime, true).getWriteStatuses(); + return UtilHelpers.handleErrors(jsc, cfg.clusteringInstantTime, writeResponse); + } } @TestOnly @@ -163,12 +163,12 @@ public Option doSchedule() throws Exception { private Option doSchedule(JavaSparkContext jsc) throws Exception { String schemaStr = getSchemaFromLatestInstant(); - SparkRDDWriteClient client = - UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props); - if (cfg.clusteringInstantTime != null) { - client.scheduleClusteringAtInstant(cfg.clusteringInstantTime, Option.empty()); - return Option.of(cfg.clusteringInstantTime); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + if (cfg.clusteringInstantTime != null) { + client.scheduleClusteringAtInstant(cfg.clusteringInstantTime, Option.empty()); + return Option.of(cfg.clusteringInstantTime); + } + return client.scheduleClustering(Option.empty()); } - return client.scheduleClustering(Option.empty()); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index ee31d4a600a22..42ab6ca558b64 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -85,7 +85,7 @@ public interface Partitioner { public static class OutputFormatValidator implements IValueValidator { public static final String HUDI = "hudi"; - public static final List FORMATS = CollectionUtils.createImmutableList("json", "parquet", HUDI); + public static final List FORMATS = CollectionUtils.createImmutableList("json", "parquet", "orc", HUDI); @Override public void validate(String name, String value) { @@ -104,7 +104,7 @@ public static class Config implements Serializable { @Parameter(names = {"--target-output-path"}, description = "Base path for the target output files (snapshots)", required = true) public String targetOutputPath; - @Parameter(names = {"--output-format"}, description = "Output format for the exported dataset; accept these values: json|parquet|hudi", required = true, + @Parameter(names = {"--output-format"}, description = "Output format for the exported dataset; accept these values: json|parquet|orc|hudi", required = true, validateValueWith = OutputFormatValidator.class) public String outputFormat; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/SourceCommitCallback.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/SourceCommitCallback.java new file mode 100644 index 0000000000000..3d3597320467c --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/callback/SourceCommitCallback.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.callback; + +/** + * A callback interface that provides the Source an option to perform action on successful Hudi commit. + */ +public interface SourceCommitCallback { + + /** + * Performs some action on successful Hudi commit like committing offsets to Kafka. + * + * @param lastCkptStr last checkpoint string. + */ + default void onCommit(String lastCkptStr) { + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index c60449a797688..cdea9120cb7f9 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -473,7 +473,7 @@ private Pair, JavaRDD> writeToSink(JavaRDD> fetchNewDataInRowFormat(Option lastCkptS throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")"); } } + + public Source getSource() { + return source; + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java index 5d0a116cab945..da04b632a977b 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java @@ -24,14 +24,11 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.kafka.common.errors.SerializationException; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; /** * Extending {@link KafkaAvroSchemaDeserializer} as we need to be able to inject reader schema during deserialization. @@ -51,9 +48,7 @@ public void configure(Map configs, boolean isKey) { super.configure(configs, isKey); try { TypedProperties props = getConvertToTypedProperties(configs); - String className = props.getString(DataSourceWriteOptions.SCHEMA_PROVIDER_CLASS_PROP()); - SchemaProvider schemaProvider = (SchemaProvider) ReflectionUtils.loadClass(className, props); - sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema(); + sourceSchema = new Schema.Parser().parse(props.getString(DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA())); } catch (Throwable e) { throw new HoodieException(e); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java index 511a72c280e8a..652e442a89d21 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java @@ -40,6 +40,9 @@ import org.apache.spark.streaming.kafka010.LocationStrategies; import org.apache.spark.streaming.kafka010.OffsetRange; +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET; +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; + /** * Reads avro serialized Kafka data, based on the confluent schema-registry. */ @@ -63,11 +66,11 @@ public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, Spa props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, KafkaAvroDeserializer.class); } else { try { + props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName)); if (schemaProvider == null) { throw new HoodieIOException("SchemaProvider has to be set to use custom Deserializer"); } - props.put(DataSourceWriteOptions.SCHEMA_PROVIDER_CLASS_PROP(), schemaProvider.getClass().getName()); - props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName)); + props.put(DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER_SCHEMA(), schemaProvider.getSourceSchema().toString()); } catch (ClassNotFoundException e) { String error = "Could not load custom avro kafka deserializer: " + deserializerClassName; LOG.error(error); @@ -95,4 +98,11 @@ private JavaRDD toRDD(OffsetRange[] offsetRanges) { return KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, LocationStrategies.PreferConsistent()).map(obj -> (GenericRecord) obj.value()); } + + @Override + public void onCommit(String lastCkptStr) { + if (this.props.getBoolean(ENABLE_KAFKA_COMMIT_OFFSET, DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET)) { + offsetGen.commitOffsetToKafka(lastCkptStr); + } + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java index cedaba48c2e3d..c1e2e3dad97ce 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java @@ -35,6 +35,9 @@ import org.apache.spark.streaming.kafka010.LocationStrategies; import org.apache.spark.streaming.kafka010.OffsetRange; +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET; +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; + /** * Read json kafka data. */ @@ -71,4 +74,11 @@ private JavaRDD toRDD(OffsetRange[] offsetRanges) { return KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, LocationStrategies.PreferConsistent()).map(x -> (String) x.value()); } + + @Override + public void onCommit(String lastCkptStr) { + if (this.props.getBoolean(ENABLE_KAFKA_COMMIT_OFFSET, DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET)) { + offsetGen.commitOffsetToKafka(lastCkptStr); + } + } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java index 4d25d479a774a..6d610d5c8cbdc 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/Source.java @@ -23,6 +23,7 @@ import org.apache.hudi.PublicAPIMethod; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; +import org.apache.hudi.utilities.callback.SourceCommitCallback; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.spark.api.java.JavaSparkContext; @@ -34,7 +35,7 @@ * Represents a source from which we can tail data. Assumes a constructor that takes properties. */ @PublicAPIClass(maturity = ApiMaturityLevel.STABLE) -public abstract class Source implements Serializable { +public abstract class Source implements SourceCommitCallback, Serializable { public enum SourceType { JSON, AVRO, ROW diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java index c14046ee42416..99e960cf55fa5 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java @@ -26,9 +26,13 @@ import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.kafka.clients.consumer.CommitFailedException; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.streaming.kafka010.OffsetRange; @@ -147,7 +151,7 @@ public static long totalNewMessages(OffsetRange[] ranges) { * Kafka reset offset strategies. */ enum KafkaResetOffsetStrategies { - LATEST, EARLIEST + LATEST, EARLIEST, GROUP } /** @@ -157,6 +161,8 @@ public static class Config { private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic"; private static final String MAX_EVENTS_FROM_KAFKA_SOURCE_PROP = "hoodie.deltastreamer.kafka.source.maxEvents"; + public static final String ENABLE_KAFKA_COMMIT_OFFSET = "hoodie.deltastreamer.source.kafka.enable.commit.offset"; + public static final Boolean DEFAULT_ENABLE_KAFKA_COMMIT_OFFSET = false; // "auto.offset.reset" is kafka native config param. Do not change the config param name. public static final String KAFKA_AUTO_OFFSET_RESET = "auto.offset.reset"; private static final KafkaResetOffsetStrategies DEFAULT_KAFKA_AUTO_OFFSET_RESET = KafkaResetOffsetStrategies.LATEST; @@ -164,22 +170,14 @@ public static class Config { public static long maxEventsFromKafkaSource = DEFAULT_MAX_EVENTS_FROM_KAFKA_SOURCE; } - private final HashMap kafkaParams; + private final Map kafkaParams; private final TypedProperties props; protected final String topicName; private KafkaResetOffsetStrategies autoResetValue; public KafkaOffsetGen(TypedProperties props) { this.props = props; - - kafkaParams = new HashMap<>(); - props.keySet().stream().filter(prop -> { - // In order to prevent printing unnecessary warn logs, here filter out the hoodie - // configuration items before passing to kafkaParams - return !prop.toString().startsWith("hoodie."); - }).forEach(prop -> { - kafkaParams.put(prop.toString(), props.get(prop.toString())); - }); + kafkaParams = excludeHoodieConfigs(props); DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME)); topicName = props.getString(Config.KAFKA_TOPIC_NAME); String kafkaAutoResetOffsetsStr = props.getString(Config.KAFKA_AUTO_OFFSET_RESET, Config.DEFAULT_KAFKA_AUTO_OFFSET_RESET.name().toLowerCase()); @@ -194,6 +192,9 @@ public KafkaOffsetGen(TypedProperties props) { if (!found) { throw new HoodieDeltaStreamerException(Config.KAFKA_AUTO_OFFSET_RESET + " config set to unknown value " + kafkaAutoResetOffsetsStr); } + if (autoResetValue.equals(KafkaResetOffsetStrategies.GROUP)) { + this.kafkaParams.put(Config.KAFKA_AUTO_OFFSET_RESET, Config.DEFAULT_KAFKA_AUTO_OFFSET_RESET.name().toLowerCase()); + } } public OffsetRange[] getNextOffsetRanges(Option lastCheckpointStr, long sourceLimit, HoodieDeltaStreamerMetrics metrics) { @@ -222,8 +223,11 @@ public OffsetRange[] getNextOffsetRanges(Option lastCheckpointStr, long case LATEST: fromOffsets = consumer.endOffsets(topicPartitions); break; + case GROUP: + fromOffsets = getGroupOffsets(consumer, topicPartitions); + break; default: - throw new HoodieNotSupportedException("Auto reset value must be one of 'earliest' or 'latest' "); + throw new HoodieNotSupportedException("Auto reset value must be one of 'earliest' or 'latest' or 'group' "); } } @@ -297,7 +301,50 @@ public String getTopicName() { return topicName; } - public HashMap getKafkaParams() { + public Map getKafkaParams() { + return kafkaParams; + } + + private Map excludeHoodieConfigs(TypedProperties props) { + Map kafkaParams = new HashMap<>(); + props.keySet().stream().filter(prop -> { + // In order to prevent printing unnecessary warn logs, here filter out the hoodie + // configuration items before passing to kafkaParams + return !prop.toString().startsWith("hoodie."); + }).forEach(prop -> { + kafkaParams.put(prop.toString(), props.get(prop.toString())); + }); return kafkaParams; } + + /** + * Commit offsets to Kafka only after hoodie commit is successful. + * @param checkpointStr checkpoint string containing offsets. + */ + public void commitOffsetToKafka(String checkpointStr) { + DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(ConsumerConfig.GROUP_ID_CONFIG)); + Map offsetMap = CheckpointUtils.strToOffsets(checkpointStr); + Map offsetAndMetadataMap = new HashMap<>(offsetMap.size()); + try (KafkaConsumer consumer = new KafkaConsumer(kafkaParams)) { + offsetMap.forEach((topicPartition, offset) -> offsetAndMetadataMap.put(topicPartition, new OffsetAndMetadata(offset))); + consumer.commitSync(offsetAndMetadataMap); + } catch (CommitFailedException | TimeoutException e) { + LOG.warn("Committing offsets to Kafka failed, this does not impact processing of records", e); + } + } + + private Map getGroupOffsets(KafkaConsumer consumer, Set topicPartitions) { + Map fromOffsets = new HashMap<>(); + for (TopicPartition topicPartition : topicPartitions) { + OffsetAndMetadata committedOffsetAndMetadata = consumer.committed(topicPartition); + if (committedOffsetAndMetadata != null) { + fromOffsets.put(topicPartition, committedOffsetAndMetadata.offset()); + } else { + LOG.warn("There are no commits associated with this consumer group, starting to consume from latest offset"); + fromOffsets = consumer.endOffsets(topicPartitions); + break; + } + } + return fromOffsets; + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieSnapshotExporter.java index ae74c1d04f109..1096c929b9b53 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieSnapshotExporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieSnapshotExporter.java @@ -32,7 +32,7 @@ public class TestHoodieSnapshotExporter { @ParameterizedTest - @ValueSource(strings = {"json", "parquet", "hudi"}) + @ValueSource(strings = {"json", "parquet", "orc", "hudi"}) public void testValidateOutputFormatWithValidFormat(String format) { assertDoesNotThrow(() -> { new OutputFormatValidator().validate(null, format); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java index 14c5f01079d88..714d636bda523 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deser/TestKafkaAvroSchemaDeserializer.java @@ -51,13 +51,12 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase { private final String topic; private final Schema origSchema = createUserSchema(); private final Schema evolSchema = createExtendUserSchema(); - private Properties defaultConfig = new Properties(); + private Properties config = new Properties(); public TestKafkaAvroSchemaDeserializer() { - defaultConfig.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "bogus"); - defaultConfig.put("hoodie.deltastreamer.schemaprovider.class", SchemaTestProvider.class.getName()); + config.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "bogus"); schemaRegistry = new MockSchemaRegistryClient(); - avroSerializer = new KafkaAvroSerializer(schemaRegistry, new HashMap(defaultConfig)); + avroSerializer = new KafkaAvroSerializer(schemaRegistry, new HashMap(config)); topic = "test"; } @@ -66,8 +65,7 @@ private Schema createUserSchema() { + "\"name\": \"User\"," + "\"fields\": [{\"name\": \"name\", \"type\": \"string\"}]}"; Schema.Parser parser = new Schema.Parser(); - Schema schema = parser.parse(userSchema); - return schema; + return parser.parse(userSchema); } private IndexedRecord createUserRecord() { @@ -83,8 +81,7 @@ private Schema createExtendUserSchema() { + "\"fields\": [{\"name\": \"name\", \"type\": \"string\"}, " + "{\"name\": \"age\", \"type\": [\"null\", \"int\"], \"default\": null}]}"; Schema.Parser parser = new Schema.Parser(); - Schema schema = parser.parse(userSchema); - return schema; + return parser.parse(userSchema); } private IndexedRecord createExtendUserRecord() { @@ -102,9 +99,10 @@ private IndexedRecord createExtendUserRecord() { public void testKafkaAvroSchemaDeserializer() { byte[] bytesOrigRecord; IndexedRecord avroRecord = createUserRecord(); - SchemaTestProvider.schemaToReturn.set(origSchema); - KafkaAvroSchemaDeserializer avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(defaultConfig)); - avroDeserializer.configure(new HashMap(defaultConfig), false); + config.put("hoodie.deltastreamer.source.kafka.value.deserializer.schema", origSchema.toString()); + + KafkaAvroSchemaDeserializer avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(config)); + avroDeserializer.configure(new HashMap(config), false); bytesOrigRecord = avroSerializer.serialize(topic, avroRecord); // record is serialized in orig schema and deserialized using same schema. assertEquals(avroRecord, avroDeserializer.deserialize(false, topic, false, bytesOrigRecord, origSchema)); @@ -113,8 +111,9 @@ public void testKafkaAvroSchemaDeserializer() { byte[] bytesExtendedRecord = avroSerializer.serialize(topic, avroRecordWithAllField); SchemaTestProvider.schemaToReturn.set(evolSchema); - avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(defaultConfig)); - avroDeserializer.configure(new HashMap(defaultConfig), false); + config.put("hoodie.deltastreamer.source.kafka.value.deserializer.schema", evolSchema.toString()); + avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(config)); + avroDeserializer.configure(new HashMap(config), false); // record is serialized w/ evolved schema, and deserialized w/ evolved schema IndexedRecord avroRecordWithAllFieldActual = (IndexedRecord) avroDeserializer.deserialize(false, topic, false, bytesExtendedRecord, evolSchema); assertEquals(avroRecordWithAllField, avroRecordWithAllFieldActual); 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 64fc5313a17fa..043fd4b8ccde8 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 @@ -971,7 +971,7 @@ private void deltaStreamerTestRunner(HoodieDeltaStreamer ds, HoodieDeltaStreamer } }); - TestHelpers.waitTillCondition(condition, dsFuture, 240); + TestHelpers.waitTillCondition(condition, dsFuture, 360); ds.shutdownGracefully(); dsFuture.get(); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java index aefa49fa90922..133205cb8a77b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java @@ -213,7 +213,7 @@ public void testExportDatasetWithNoPartition() throws IOException { public class TestHoodieSnapshotExporterForNonHudi { @ParameterizedTest - @ValueSource(strings = {"json", "parquet"}) + @ValueSource(strings = {"json", "parquet", "orc"}) public void testExportAsNonHudi(String format) throws IOException { HoodieSnapshotExporter.Config cfg = new Config(); cfg.sourceBasePath = sourcePath; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java index ec8a945c41065..a1a00faa592a6 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; @@ -30,6 +31,9 @@ import org.apache.avro.generic.GenericRecord; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -40,9 +44,15 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import java.util.UUID; +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.ENABLE_KAFKA_COMMIT_OFFSET; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; /** @@ -58,7 +68,7 @@ public class TestKafkaSource extends UtilitiesTestBase { @BeforeAll public static void initClass() throws Exception { - UtilitiesTestBase.initClass(); + UtilitiesTestBase.initClass(false); } @AfterAll @@ -85,6 +95,7 @@ private TypedProperties createPropsForJsonSource(Long maxEventsToReadFromKafkaSo props.setProperty("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME); props.setProperty("bootstrap.servers", testUtils.brokerAddress()); props.setProperty(Config.KAFKA_AUTO_OFFSET_RESET, resetStrategy); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); props.setProperty("hoodie.deltastreamer.kafka.source.maxEvents", maxEventsToReadFromKafkaSource != null ? String.valueOf(maxEventsToReadFromKafkaSource) : String.valueOf(Config.maxEventsFromKafkaSource)); @@ -276,4 +287,64 @@ public void testJsonKafkaSourceWithConfigurableUpperCap() { kafkaSource.fetchNewDataInAvroFormat(Option.of(fetch4.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(Option.empty(), fetch6.getBatch()); } + + @Test + public void testCommitOffsetToKafka() { + // topic setup. + testUtils.createTopic(TEST_TOPIC_NAME, 2); + List topicPartitions = new ArrayList<>(); + TopicPartition topicPartition0 = new TopicPartition(TEST_TOPIC_NAME, 0); + topicPartitions.add(topicPartition0); + TopicPartition topicPartition1 = new TopicPartition(TEST_TOPIC_NAME, 1); + topicPartitions.add(topicPartition1); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = createPropsForJsonSource(null, "earliest"); + props.put(ENABLE_KAFKA_COMMIT_OFFSET, "true"); + Source jsonSource = new JsonKafkaSource(props, jsc, sparkSession, schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); + + // 1. Extract without any checkpoint => get all the data, respecting sourceLimit + assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 599); + // commit to kafka after first batch + kafkaSource.getSource().onCommit(fetch1.getCheckpointForNextBatch()); + try (KafkaConsumer consumer = new KafkaConsumer(props)) { + consumer.assign(topicPartitions); + + OffsetAndMetadata offsetAndMetadata = consumer.committed(topicPartition0); + assertNotNull(offsetAndMetadata); + assertEquals(300, offsetAndMetadata.offset()); + offsetAndMetadata = consumer.committed(topicPartition1); + assertNotNull(offsetAndMetadata); + assertEquals(299, offsetAndMetadata.offset()); + // end offsets will point to 500 for each partition because we consumed less messages from first batch + Map endOffsets = consumer.endOffsets(topicPartitions); + assertEquals(500L, endOffsets.get(topicPartition0)); + assertEquals(500L, endOffsets.get(topicPartition1)); + + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 500))); + InputBatch> fetch2 = + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + + // commit to Kafka after second batch is processed completely + kafkaSource.getSource().onCommit(fetch2.getCheckpointForNextBatch()); + + offsetAndMetadata = consumer.committed(topicPartition0); + assertNotNull(offsetAndMetadata); + assertEquals(750, offsetAndMetadata.offset()); + offsetAndMetadata = consumer.committed(topicPartition1); + assertNotNull(offsetAndMetadata); + assertEquals(750, offsetAndMetadata.offset()); + + endOffsets = consumer.endOffsets(topicPartitions); + assertEquals(750L, endOffsets.get(topicPartition0)); + assertEquals(750L, endOffsets.get(topicPartition1)); + } + // check failure case + props.remove(ConsumerConfig.GROUP_ID_CONFIG); + assertThrows(HoodieNotSupportedException.class,() -> kafkaSource.getSource().onCommit("")); + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestKafkaOffsetGen.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestKafkaOffsetGen.java new file mode 100644 index 0000000000000..ccc141b51a491 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestKafkaOffsetGen.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.helpers; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieNotSupportedException; +import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config; +import org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.spark.streaming.kafka010.KafkaTestUtils; +import org.apache.spark.streaming.kafka010.OffsetRange; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +/** + * Tests against {@link KafkaOffsetGen}. + */ +public class TestKafkaOffsetGen { + + private static String TEST_TOPIC_NAME = "hoodie_test"; + private KafkaTestUtils testUtils; + private HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); + + @BeforeEach + public void setup() throws Exception { + testUtils = new KafkaTestUtils(); + testUtils.setup(); + } + + @AfterEach + public void teardown() throws Exception { + testUtils.teardown(); + } + + private TypedProperties getConsumerConfigs(String autoOffsetReset) { + TypedProperties props = new TypedProperties(); + props.put(Config.KAFKA_AUTO_OFFSET_RESET, autoOffsetReset); + props.put("hoodie.deltastreamer.source.kafka.topic", TEST_TOPIC_NAME); + props.setProperty("bootstrap.servers", testUtils.brokerAddress()); + props.setProperty("key.deserializer", StringDeserializer.class.getName()); + props.setProperty("value.deserializer", StringDeserializer.class.getName()); + props.setProperty(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + return props; + } + + @Test + public void testGetNextOffsetRangesFromEarliest() { + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + testUtils.createTopic(TEST_TOPIC_NAME, 1); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("earliest")); + OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 500, metrics); + assertEquals(1, nextOffsetRanges.length); + assertEquals(0, nextOffsetRanges[0].fromOffset()); + assertEquals(500, nextOffsetRanges[0].untilOffset()); + + nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 5000, metrics); + assertEquals(1, nextOffsetRanges.length); + assertEquals(0, nextOffsetRanges[0].fromOffset()); + assertEquals(1000, nextOffsetRanges[0].untilOffset()); + } + + @Test + public void testGetNextOffsetRangesFromLatest() { + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + testUtils.createTopic(TEST_TOPIC_NAME, 1); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("latest")); + OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 500, metrics); + assertEquals(1, nextOffsetRanges.length); + assertEquals(1000, nextOffsetRanges[0].fromOffset()); + assertEquals(1000, nextOffsetRanges[0].untilOffset()); + } + + @Test + public void testGetNextOffsetRangesFromCheckpoint() { + String lastCheckpointString = TEST_TOPIC_NAME + ",0:250"; + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + testUtils.createTopic(TEST_TOPIC_NAME, 1); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("latest")); + + OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.of(lastCheckpointString), 500, metrics); + assertEquals(1, nextOffsetRanges.length); + assertEquals(250, nextOffsetRanges[0].fromOffset()); + assertEquals(750, nextOffsetRanges[0].untilOffset()); + } + + @Test + public void testGetNextOffsetRangesFromMultiplePartitions() { + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + testUtils.createTopic(TEST_TOPIC_NAME, 2); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("earliest")); + OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 499, metrics); + assertEquals(2, nextOffsetRanges.length); + assertEquals(0, nextOffsetRanges[0].fromOffset()); + assertEquals(250, nextOffsetRanges[0].untilOffset()); + assertEquals(0, nextOffsetRanges[1].fromOffset()); + assertEquals(249, nextOffsetRanges[1].untilOffset()); + } + + @Test + public void testGetNextOffsetRangesFromGroup() { + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + testUtils.createTopic(TEST_TOPIC_NAME, 2); + testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("group")); + String lastCheckpointString = TEST_TOPIC_NAME + ",0:250,1:249"; + kafkaOffsetGen.commitOffsetToKafka(lastCheckpointString); + // don't pass lastCheckpointString as we want to read from group committed offset + OffsetRange[] nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 300, metrics); + assertEquals(250, nextOffsetRanges[0].fromOffset()); + assertEquals(400, nextOffsetRanges[0].untilOffset()); + assertEquals(249, nextOffsetRanges[1].fromOffset()); + assertEquals(399, nextOffsetRanges[1].untilOffset()); + + // committed offsets are not present for the consumer group + kafkaOffsetGen = new KafkaOffsetGen(getConsumerConfigs("group")); + nextOffsetRanges = kafkaOffsetGen.getNextOffsetRanges(Option.empty(), 300, metrics); + assertEquals(500, nextOffsetRanges[0].fromOffset()); + assertEquals(500, nextOffsetRanges[0].untilOffset()); + assertEquals(500, nextOffsetRanges[1].fromOffset()); + assertEquals(500, nextOffsetRanges[1].untilOffset()); + } + + @Test + public void testCheckTopicExists() { + TypedProperties props = getConsumerConfigs("latest"); + KafkaOffsetGen kafkaOffsetGen = new KafkaOffsetGen(props); + testUtils.createTopic(TEST_TOPIC_NAME, 1); + boolean topicExists = kafkaOffsetGen.checkTopicExists(new KafkaConsumer(props)); + assertTrue(topicExists); + props.put("hoodie.deltastreamer.source.kafka.topic", "random"); + kafkaOffsetGen = new KafkaOffsetGen(props); + topicExists = kafkaOffsetGen.checkTopicExists(new KafkaConsumer(props)); + assertFalse(topicExists); + } + + @Test + public void testTopicNameNotPresentInProps() { + assertThrows(HoodieNotSupportedException.class, () -> new KafkaOffsetGen(new TypedProperties())); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java index 524591dd7bca5..0dbf31ec68537 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/sources/AbstractBaseTestSource.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.RawTripTestPayload; -import org.apache.hudi.common.util.collection.RocksDBBasedMap; +import org.apache.hudi.common.util.collection.RocksDbDiskMap; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.AvroSource; @@ -62,7 +62,7 @@ public static void initDataGen(TypedProperties props, int partition) { File.createTempFile("test_data_gen", ".keys").getParent()) + "/" + partition; LOG.info("useRocksForTestDataGenKeys=" + useRocksForTestDataGenKeys + ", BaseStoreDir=" + baseStoreDir); dataGeneratorMap.put(partition, new HoodieTestDataGenerator(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, - useRocksForTestDataGenKeys ? new RocksDBBasedMap<>(baseStoreDir) : new HashMap<>())); + useRocksForTestDataGenKeys ? new RocksDbDiskMap<>(baseStoreDir) : new HashMap<>())); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); }