diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index e05ac87bc3fae..1267fd01065bd 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -6,6 +6,15 @@ on: - master - 'release-*' pull_request: + paths-ignore: + - '**.bmp' + - '**.gif' + - '**.jpg' + - '**.jpeg' + - '**.md' + - '**.pdf' + - '**.png' + - '**.svg' branches: - master - 'release-*' diff --git a/.gitignore b/.gitignore index 2983889cc6e17..ff099d52067e9 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,7 @@ # Directories # /build/ target/ +metastore_db/ .mvn/ # OS Files # diff --git a/azure-pipelines.yml b/azure-pipelines.yml index d450bfb9ab2b0..c0444c1d1f0ab 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -19,7 +19,7 @@ trigger: - '*' # must quote since "*" is a YAML reserved character; we want a string pool: - vmImage: 'ubuntu-18.04' + vmImage: 'ubuntu-22.04' parameters: - name: job1Modules @@ -46,23 +46,7 @@ parameters: - name: job4Modules type: object default: - - '!hudi-client/hudi-spark-client' - - '!hudi-common' - - '!hudi-examples' - - '!hudi-examples/hudi-examples-common' - - '!hudi-examples/hudi-examples-flink' - - '!hudi-examples/hudi-examples-java' - - '!hudi-examples/hudi-examples-spark' - - '!hudi-flink-datasource' - - '!hudi-flink-datasource/hudi-flink' - - '!hudi-flink-datasource/hudi-flink1.13.x' - - '!hudi-flink-datasource/hudi-flink1.14.x' - - '!hudi-flink-datasource/hudi-flink1.15.x' - - '!hudi-spark-datasource' - - '!hudi-spark-datasource/hudi-spark' - - '!hudi-spark-datasource/hudi-spark2' - - '!hudi-spark-datasource/hudi-spark2-common' - - '!hudi-spark-datasource/hudi-spark-common' + - 'hudi-utilities' variables: BUILD_PROFILES: '-Dscala-2.11 -Dspark2.4 -Dflink1.14' @@ -80,96 +64,6 @@ variables: stages: - stage: test jobs: - - job: UT_FT_1 - displayName: UT FT common & flink & UT client/spark-client - timeoutInMinutes: '150' - steps: - - task: Maven@4 - displayName: maven install - inputs: - mavenPomFile: 'pom.xml' - goals: 'clean install' - options: $(MVN_OPTS_INSTALL) - publishJUnitResults: false - jdkVersionOption: '1.8' - - task: Maven@4 - displayName: UT common flink client/spark-client - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Punit-tests -pl $(JOB1_MODULES),hudi-client/hudi-spark-client - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - task: Maven@4 - displayName: FT common flink - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Pfunctional-tests -pl $(JOB1_MODULES) - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - script: | - grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 - displayName: Top 100 long-running testcases - - job: UT_FT_2 - displayName: FT client/spark-client - timeoutInMinutes: '150' - steps: - - task: Maven@4 - displayName: maven install - inputs: - mavenPomFile: 'pom.xml' - goals: 'clean install' - options: $(MVN_OPTS_INSTALL) - publishJUnitResults: false - jdkVersionOption: '1.8' - - task: Maven@4 - displayName: FT client/spark-client - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Pfunctional-tests -pl $(JOB2_MODULES) - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - script: | - grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 - displayName: Top 100 long-running testcases - - job: UT_FT_3 - displayName: UT FT spark-datasource - timeoutInMinutes: '150' - steps: - - task: Maven@4 - displayName: maven install - inputs: - mavenPomFile: 'pom.xml' - goals: 'clean install' - options: $(MVN_OPTS_INSTALL) - publishJUnitResults: false - jdkVersionOption: '1.8' - - task: Maven@4 - displayName: UT spark-datasource - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Punit-tests -pl $(JOB3_MODULES) - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - task: Maven@4 - displayName: FT spark-datasource - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Pfunctional-tests -pl $(JOB3_MODULES) - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - script: | - grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 - displayName: Top 100 long-running testcases - job: UT_FT_4 displayName: UT FT other modules timeoutInMinutes: '150' @@ -191,51 +85,6 @@ stages: publishJUnitResults: false jdkVersionOption: '1.8' mavenOptions: '-Xmx4g' - - task: Maven@4 - displayName: FT other modules - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Pfunctional-tests -pl $(JOB4_MODULES) - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - script: | grep "testcase" */target/surefire-reports/*.xml */*/target/surefire-reports/*.xml | awk -F'"' ' { print $6,$4,$2 } ' | sort -nr | head -n 100 - displayName: Top 100 long-running testcases - - job: IT - displayName: IT modules - timeoutInMinutes: '150' - steps: - - task: Maven@4 - displayName: maven install - inputs: - mavenPomFile: 'pom.xml' - goals: 'clean install' - options: $(MVN_OPTS_INSTALL) -Pintegration-tests - publishJUnitResults: false - jdkVersionOption: '1.8' - - task: Maven@4 - displayName: UT integ-test - inputs: - mavenPomFile: 'pom.xml' - goals: 'test' - options: $(MVN_OPTS_TEST) -Pintegration-tests -DskipUTs=false -DskipITs=true -pl hudi-integ-test - publishJUnitResults: false - jdkVersionOption: '1.8' - mavenOptions: '-Xmx4g' - - task: AzureCLI@2 - displayName: Prepare for IT - inputs: - azureSubscription: apachehudici-service-connection - scriptType: bash - scriptLocation: inlineScript - inlineScript: | - echo 'Downloading $(SPARK_ARCHIVE)' - az storage blob download -c ci-caches -n $(SPARK_ARCHIVE).tgz -f $(Pipeline.Workspace)/$(SPARK_ARCHIVE).tgz --account-name apachehudici - tar -xvf $(Pipeline.Workspace)/$(SPARK_ARCHIVE).tgz -C $(Pipeline.Workspace)/ - mkdir /tmp/spark-events/ - - script: | - export SPARK_HOME=$(Pipeline.Workspace)/$(SPARK_ARCHIVE) - mvn $(MVN_OPTS_TEST) -Pintegration-tests verify - displayName: IT + displayName: Top 100 long-running testcases \ No newline at end of file diff --git a/doap_HUDI.rdf b/doap_HUDI.rdf index e153fb3d4c5fc..0817e38f99318 100644 --- a/doap_HUDI.rdf +++ b/doap_HUDI.rdf @@ -101,6 +101,16 @@ 2022-08-16 0.12.0 + + Apache Hudi 0.12.1 + 2022-10-18 + 0.12.1 + + + Apache Hudi 0.12.2 + 2022-12-28 + 0.12.2 + diff --git a/docker/demo/config/test-suite/compact-test.properties b/docker/demo/config/test-suite/compact-test.properties index 2eca88de3a426..f87d20974c6c2 100644 --- a/docker/demo/config/test-suite/compact-test.properties +++ b/docker/demo/config/test-suite/compact-test.properties @@ -22,7 +22,7 @@ hoodie.bulkinsert.shuffle.parallelism=100 hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-1.properties b/docker/demo/config/test-suite/multi-writer-1.properties index 502a1b771e8cd..692b4f911dfa3 100644 --- a/docker/demo/config/test-suite/multi-writer-1.properties +++ b/docker/demo/config/test-suite/multi-writer-1.properties @@ -25,7 +25,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-2.properties b/docker/demo/config/test-suite/multi-writer-2.properties index 80db8912b5406..6b9915ef95b42 100644 --- a/docker/demo/config/test-suite/multi-writer-2.properties +++ b/docker/demo/config/test-suite/multi-writer-2.properties @@ -25,7 +25,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-local-1.properties b/docker/demo/config/test-suite/multi-writer-local-1.properties index be16f91c17459..679067fa0e446 100644 --- a/docker/demo/config/test-suite/multi-writer-local-1.properties +++ b/docker/demo/config/test-suite/multi-writer-local-1.properties @@ -28,7 +28,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-local-2.properties b/docker/demo/config/test-suite/multi-writer-local-2.properties index 08f294ce1461e..2a70bf138b5e0 100644 --- a/docker/demo/config/test-suite/multi-writer-local-2.properties +++ b/docker/demo/config/test-suite/multi-writer-local-2.properties @@ -28,7 +28,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-local-3.properties b/docker/demo/config/test-suite/multi-writer-local-3.properties index 48f0f0b1ace8b..89b02b02f44c2 100644 --- a/docker/demo/config/test-suite/multi-writer-local-3.properties +++ b/docker/demo/config/test-suite/multi-writer-local-3.properties @@ -28,7 +28,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/multi-writer-local-4.properties b/docker/demo/config/test-suite/multi-writer-local-4.properties index 4b5120928ccb1..7a6a57487a69a 100644 --- a/docker/demo/config/test-suite/multi-writer-local-4.properties +++ b/docker/demo/config/test-suite/multi-writer-local-4.properties @@ -28,7 +28,7 @@ hoodie.metadata.enable=false hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector diff --git a/docker/demo/config/test-suite/templates/test.properties.template b/docker/demo/config/test-suite/templates/test.properties.template index e1b65fb730a18..1c84be89daef2 100644 --- a/docker/demo/config/test-suite/templates/test.properties.template +++ b/docker/demo/config/test-suite/templates/test.properties.template @@ -22,7 +22,7 @@ hoodie.bulkinsert.shuffle.parallelism=100 hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false hoodie.deltastreamer.source.test.max_unique_records=100000000 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.input.selector=org.apache.hudi.integ.testsuite.helpers.DFSTestSuitePathSelector hoodie.datasource.hive_sync.skip_ro_suffix=true diff --git a/docker/demo/config/test-suite/test-aggressive-clean-archival-inline-compact.properties b/docker/demo/config/test-suite/test-aggressive-clean-archival-inline-compact.properties index 14427f323cead..c8b8820859cc1 100644 --- a/docker/demo/config/test-suite/test-aggressive-clean-archival-inline-compact.properties +++ b/docker/demo/config/test-suite/test-aggressive-clean-archival-inline-compact.properties @@ -28,7 +28,7 @@ hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-aggressive-clean-archival.properties b/docker/demo/config/test-suite/test-aggressive-clean-archival.properties index f0d9de251b869..35c6f89d8774a 100644 --- a/docker/demo/config/test-suite/test-aggressive-clean-archival.properties +++ b/docker/demo/config/test-suite/test-aggressive-clean-archival.properties @@ -26,7 +26,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.cleaner.commits.retained=8 hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival-inline-compact.properties b/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival-inline-compact.properties index 748972861851d..fc63994250409 100644 --- a/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival-inline-compact.properties +++ b/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival-inline-compact.properties @@ -24,7 +24,7 @@ hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.cleaner.commits.retained=8 hoodie.keep.min.commits=12 diff --git a/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival.properties b/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival.properties index b94ccabb55e09..c0ee38db09dcd 100644 --- a/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival.properties +++ b/docker/demo/config/test-suite/test-clustering-aggressive-clean-archival.properties @@ -26,7 +26,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.cleaner.commits.retained=8 hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-clustering-inline-compact.properties b/docker/demo/config/test-suite/test-clustering-inline-compact.properties index 5e86790c723a9..1585138e9d8c7 100644 --- a/docker/demo/config/test-suite/test-clustering-inline-compact.properties +++ b/docker/demo/config/test-suite/test-clustering-inline-compact.properties @@ -23,7 +23,7 @@ hoodie.upsert.shuffle.parallelism=25 hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 -hoodie.embed.timeline.server=false + hoodie.compact.inline=true hoodie.deltastreamer.source.test.num_partitions=100 diff --git a/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival-inline-compact.properties b/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival-inline-compact.properties index dd3089d190184..59470745803df 100644 --- a/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival-inline-compact.properties +++ b/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival-inline-compact.properties @@ -29,7 +29,7 @@ hoodie.keep.max.commits=14 hoodie.compact.inline=true hoodie.metadata.enable=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival.properties b/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival.properties index c10d6ecc48007..ed9d140586635 100644 --- a/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival.properties +++ b/docker/demo/config/test-suite/test-clustering-metadata-aggressive-clean-archival.properties @@ -27,7 +27,7 @@ hoodie.cleaner.commits.retained=8 hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 -hoodie.embed.timeline.server=false + hoodie.metadata.enable=true hoodie.deltastreamer.source.test.num_partitions=100 diff --git a/docker/demo/config/test-suite/test-clustering.properties b/docker/demo/config/test-suite/test-clustering.properties index 677cf96751d77..4fb5fd858100c 100644 --- a/docker/demo/config/test-suite/test-clustering.properties +++ b/docker/demo/config/test-suite/test-clustering.properties @@ -23,7 +23,7 @@ hoodie.upsert.shuffle.parallelism=25 hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-inline-compact.properties b/docker/demo/config/test-suite/test-inline-compact.properties index 76de6bd2678e1..330d242deba50 100644 --- a/docker/demo/config/test-suite/test-inline-compact.properties +++ b/docker/demo/config/test-suite/test-inline-compact.properties @@ -25,7 +25,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=false hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival-inline-compact.properties b/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival-inline-compact.properties index 7921162356e2d..37a01242c20ee 100644 --- a/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival-inline-compact.properties +++ b/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival-inline-compact.properties @@ -27,7 +27,7 @@ hoodie.cleaner.commits.retained=8 hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 -hoodie.embed.timeline.server=false + hoodie.metadata.enable=true hoodie.compact.inline=true diff --git a/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival.properties b/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival.properties index 5bad7fc4ef100..c535487ecba06 100644 --- a/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival.properties +++ b/docker/demo/config/test-suite/test-metadata-aggressive-clean-archival.properties @@ -28,7 +28,7 @@ hoodie.keep.min.commits=12 hoodie.keep.max.commits=14 hoodie.metadata.enable=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-metadata-inline-compact.properties b/docker/demo/config/test-suite/test-metadata-inline-compact.properties index 5230a1488ca67..14c7e85039e18 100644 --- a/docker/demo/config/test-suite/test-metadata-inline-compact.properties +++ b/docker/demo/config/test-suite/test-metadata-inline-compact.properties @@ -25,7 +25,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=true hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-metadata.properties b/docker/demo/config/test-suite/test-metadata.properties index 0edcd3c63d2ef..f79c582306d25 100644 --- a/docker/demo/config/test-suite/test-metadata.properties +++ b/docker/demo/config/test-suite/test-metadata.properties @@ -24,7 +24,7 @@ hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-nonpartitioned-inline-compact.properties b/docker/demo/config/test-suite/test-nonpartitioned-inline-compact.properties index 97f2bfa4978d3..3c47517d9cfa6 100644 --- a/docker/demo/config/test-suite/test-nonpartitioned-inline-compact.properties +++ b/docker/demo/config/test-suite/test-nonpartitioned-inline-compact.properties @@ -25,7 +25,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=false hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-nonpartitioned-metadata-inline-compact.properties b/docker/demo/config/test-suite/test-nonpartitioned-metadata-inline-compact.properties index 2298be18fe91d..2b35b802a69cd 100644 --- a/docker/demo/config/test-suite/test-nonpartitioned-metadata-inline-compact.properties +++ b/docker/demo/config/test-suite/test-nonpartitioned-metadata-inline-compact.properties @@ -25,7 +25,7 @@ hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=true hoodie.compact.inline=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-nonpartitioned-metadata.properties b/docker/demo/config/test-suite/test-nonpartitioned-metadata.properties index 520534f3b3e92..05cd0ad0a30b2 100644 --- a/docker/demo/config/test-suite/test-nonpartitioned-metadata.properties +++ b/docker/demo/config/test-suite/test-nonpartitioned-metadata.properties @@ -24,7 +24,7 @@ hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=true -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test-nonpartitioned.properties b/docker/demo/config/test-suite/test-nonpartitioned.properties index d51c4e5f843d3..a4ec7145de3a0 100644 --- a/docker/demo/config/test-suite/test-nonpartitioned.properties +++ b/docker/demo/config/test-suite/test-nonpartitioned.properties @@ -24,7 +24,7 @@ hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=false -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/docker/demo/config/test-suite/test.properties b/docker/demo/config/test-suite/test.properties index 3b20d3286251a..ce8a16e3db845 100644 --- a/docker/demo/config/test-suite/test.properties +++ b/docker/demo/config/test-suite/test.properties @@ -21,7 +21,7 @@ hoodie.bulkinsert.shuffle.parallelism=25 hoodie.delete.shuffle.parallelism=25 hoodie.metadata.enable=false -hoodie.embed.timeline.server=false + hoodie.deltastreamer.source.test.num_partitions=100 hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java index b8f0d565df7f7..fcea6e578a981 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java @@ -18,6 +18,8 @@ package org.apache.hudi.aws.sync; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; @@ -26,6 +28,8 @@ import java.util.Properties; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; + /** * Currently Experimental. Utility class that implements syncing a Hudi Table with the * AWS Glue Data Catalog (https://docs.aws.amazon.com/glue/latest/dg/populate-data-catalog.html) @@ -56,6 +60,9 @@ public static void main(String[] args) { cmd.usage(); System.exit(0); } - new AwsGlueCatalogSyncTool(params.toProps(), new Configuration()).syncHoodieTable(); + // HiveConf needs to load fs conf to allow instantiation via AWSGlueClientFactory + TypedProperties props = params.toProps(); + Configuration hadoopConf = FSUtils.getFs(props.getString(META_SYNC_BASE_PATH.key()), new Configuration()).getConf(); + new AwsGlueCatalogSyncTool(props, hadoopConf).syncHoodieTable(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index b4958f5692db4..e004d91836604 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -59,6 +59,7 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieArchivalConfig; @@ -233,8 +234,7 @@ public boolean commitStats(String instantTime, List stats, Opti try { preCommit(inflightInstant, metadata); commit(table, commitActionType, instantTime, metadata, stats); - // already within lock, and so no lock requried for archival - postCommit(table, metadata, instantTime, extraMetadata, false); + postCommit(table, metadata, instantTime, extraMetadata); LOG.info("Committed " + instantTime); releaseResources(); } catch (IOException e) { @@ -243,6 +243,9 @@ public boolean commitStats(String instantTime, List stats, Opti this.txnManager.endTransaction(Option.of(inflightInstant)); } + // trigger clean and archival. + // Each internal call should ensure to lock if required. + mayBeCleanAndArchive(table); // We don't want to fail the commit if hoodie.fail.writes.on.inline.table.service.exception is false. We catch warn if false try { // do this outside of lock since compaction, clustering can be time taking and we don't need a lock for the entire execution period @@ -317,6 +320,8 @@ private void saveInternalSchema(HoodieTable table, String instantTime, HoodieCom protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf); + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient); + void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { if (writeTimer != null) { long durationInMs = metrics.getDurationInMs(writeTimer.stop()); @@ -535,21 +540,26 @@ public void preWrite(String instantTime, WriteOperationType writeOperationType, * @param metadata Commit Metadata corresponding to committed instant * @param instantTime Instant Time * @param extraMetadata Additional Metadata passed by user - * @param acquireLockForArchival true if lock has to be acquired for archival. false otherwise. */ - protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata, - boolean acquireLockForArchival) { + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - autoCleanOnCommit(); - autoArchiveOnCommit(table, acquireLockForArchival); } finally { this.heartbeatClient.stop(instantTime); } } + /** + * Triggers cleaning and archival for the table of interest. This method is called outside of locks. So, internal callers should ensure they acquire lock whereever applicable. + * @param table instance of {@link HoodieTable} of interest. + */ + protected void mayBeCleanAndArchive(HoodieTable table) { + autoCleanOnCommit(); + autoArchiveOnCommit(table); + } + protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { if (!tableServicesEnabled(config)) { return; @@ -624,11 +634,11 @@ protected void autoCleanOnCommit() { } else { LOG.info("Start to clean synchronously."); // Do not reuse instantTime for clean as metadata table requires all changes to have unique instant timestamps. - clean(true); + clean(); } } - protected void autoArchiveOnCommit(HoodieTable table, boolean acquireLockForArchival) { + protected void autoArchiveOnCommit(HoodieTable table) { if (!config.isAutoArchive()) { return; } @@ -639,7 +649,7 @@ protected void autoArchiveOnCommit(HoodieTable table, boolean acquireLockForArch LOG.info("Async archiver has finished"); } else { LOG.info("Start to archive synchronously."); - archive(table, acquireLockForArchival); + archive(table); } } @@ -848,10 +858,12 @@ public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOExcepti * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. * @return instance of {@link HoodieCleanMetadata}. */ + @Deprecated public HoodieCleanMetadata clean(String cleanInstantTime, boolean skipLocking) throws HoodieIOException { - return clean(cleanInstantTime, true, skipLocking); + return clean(cleanInstantTime, true, false); } + /** * Clean up any stale/old files/data lying around (either on file storage or index storage) based on the * configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be @@ -868,7 +880,7 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline } final Timer.Context timerContext = metrics.getCleanCtx(); CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), - HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites(skipLocking)); + HoodieTimeline.CLEAN_ACTION, () -> rollbackFailedWrites()); HoodieTable table = createTable(config, hadoopConf); if (config.allowMultipleCleans() || !table.getActiveTimeline().getCleanerTimeline().filterInflightsAndRequested().firstInstant().isPresent()) { @@ -881,7 +893,7 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline } // Proceeds to execute any requested or inflight clean instances in the timeline - HoodieCleanMetadata metadata = table.clean(context, cleanInstantTime, skipLocking); + HoodieCleanMetadata metadata = table.clean(context, cleanInstantTime); if (timerContext != null && metadata != null) { long durationMs = metrics.getDurationInMs(timerContext.stop()); metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted()); @@ -893,7 +905,7 @@ public HoodieCleanMetadata clean(String cleanInstantTime, boolean scheduleInline } public HoodieCleanMetadata clean() { - return clean(false); + return clean(HoodieActiveTimeline.createNewInstantTime()); } /** @@ -902,24 +914,24 @@ public HoodieCleanMetadata clean() { * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. * @return instance of {@link HoodieCleanMetadata}. */ + @Deprecated public HoodieCleanMetadata clean(boolean skipLocking) { - return clean(HoodieActiveTimeline.createNewInstantTime(), skipLocking); + return clean(HoodieActiveTimeline.createNewInstantTime()); } /** * Trigger archival for the table. This ensures that the number of commits do not explode * and keep increasing unbounded over time. * @param table table to commit on. - * @param acquireLockForArchival true if lock has to be acquired for archival. false otherwise. */ - protected void archive(HoodieTable table, boolean acquireLockForArchival) { + protected void archive(HoodieTable table) { if (!tableServicesEnabled(config)) { return; } try { // We cannot have unbounded commit files. Archive commits if we have to archive HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table); - archiver.archiveIfRequired(context, acquireLockForArchival); + archiver.archiveIfRequired(context, true); } catch (IOException ioe) { throw new HoodieIOException("Failed to archive", ioe); } @@ -932,7 +944,7 @@ protected void archive(HoodieTable table, boolean acquireLockForArchival) { public void archive() { // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = createTable(config, hadoopConf); - archive(table, true); + archive(table); } /** @@ -1172,9 +1184,10 @@ protected Map> getPendingRollbackInfos } /** - * Rollback all failed writes. + * Rollback failed writes if any. + * @return true if rollback happened. false otherwise. */ - protected Boolean rollbackFailedWrites() { + public Boolean rollbackFailedWrites() { return rollbackFailedWrites(false); } @@ -1188,7 +1201,7 @@ protected Boolean rollbackFailedWrites(boolean skipLocking) { Map> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient()); instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); rollbackFailedWrites(pendingRollbacks, skipLocking); - return true; + return !pendingRollbacks.isEmpty(); } protected void rollbackFailedWrites(Map> instantsToRollback, boolean skipLocking) { @@ -1425,17 +1438,38 @@ public HoodieMetrics getMetrics() { } /** - * Instantiates engine-specific instance of {@link HoodieTable} as well as performs necessary - * bootstrapping operations (for ex, validating whether Metadata Table has to be bootstrapped) + * Performs necessary bootstrapping operations (for ex, validating whether Metadata Table has to be bootstrapped). * - * NOTE: THIS OPERATION IS EXECUTED UNDER LOCK, THEREFORE SHOULD AVOID ANY OPERATIONS - * NOT REQUIRING EXTERNAL SYNCHRONIZATION + *

NOTE: THIS OPERATION IS EXECUTED UNDER LOCK, THEREFORE SHOULD AVOID ANY OPERATIONS + * NOT REQUIRING EXTERNAL SYNCHRONIZATION * * @param metaClient instance of {@link HoodieTableMetaClient} * @param instantTime current inflight instant time - * @return instantiated {@link HoodieTable} */ - protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary); + protected void doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { + Option ownerInstant = Option.empty(); + if (instantTime.isPresent()) { + ownerInstant = Option.of(new HoodieInstant(true, CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime.get())); + } + this.txnManager.beginTransaction(ownerInstant, Option.empty()); + try { + tryUpgrade(metaClient, instantTime); + if (initialMetadataTableIfNecessary) { + initMetadataTable(instantTime); + } + } finally { + this.txnManager.endTransaction(ownerInstant); + } + } + + /** + * Bootstrap the metadata table. + * + * @param instantTime current inflight instant time + */ + protected void initMetadataTable(Option instantTime) { + // by default do nothing. + } /** * Instantiates and initializes instance of {@link HoodieTable}, performing crucial bootstrapping @@ -1457,18 +1491,8 @@ protected final HoodieTable initTable(WriteOperationType operationType, Option ownerInstant = Option.empty(); - if (instantTime.isPresent()) { - ownerInstant = Option.of(new HoodieInstant(true, CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime.get())); - } - this.txnManager.beginTransaction(ownerInstant, Option.empty()); - try { - tryUpgrade(metaClient, instantTime); - table = doInitTable(metaClient, instantTime, initialMetadataTableIfNecessary); - } finally { - this.txnManager.endTransaction(ownerInstant); - } + doInitTable(metaClient, instantTime, initialMetadataTableIfNecessary); + HoodieTable table = createTable(config, hadoopConf, metaClient); // Validate table properties metaClient.validateTableProperties(config.getProps()); @@ -1513,7 +1537,8 @@ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); - if (commitMetadata.getExtraMetadata().containsKey(SCHEMA_KEY)) { + String extraSchema = commitMetadata.getExtraMetadata().get(SCHEMA_KEY); + if (!StringUtils.isNullOrEmpty(extraSchema)) { config.setSchema(commitMetadata.getExtraMetadata().get(SCHEMA_KEY)); } else { throw new HoodieIOException("Latest commit does not have any schema in commit metadata"); @@ -1557,6 +1582,12 @@ private void setWriteTimer(HoodieTable table) { } } + /** + * Upgrades the hoodie table if need be when moving to a new Hudi version. + * This method is called within a lock. Try to avoid double locking from within this method. + * @param metaClient instance of {@link HoodieTableMetaClient} to use. + * @param instantTime instant time of interest if we have one. + */ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option instantTime) { UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); @@ -1569,7 +1600,6 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta if (!instantsToRollback.isEmpty()) { Map> pendingRollbacks = getPendingRollbackInfos(metaClient); instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); - rollbackFailedWrites(pendingRollbacks, true); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java index 2992f4abd4c9e..eaa0d8d167fbb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java @@ -43,7 +43,9 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.FileIOUtils; @@ -397,7 +399,7 @@ private Stream getCleanInstantsToArchive() { }).flatMap(Collection::stream); } - private Stream getCommitInstantsToArchive() { + private Stream getCommitInstantsToArchive() throws IOException { // TODO (na) : Add a way to return actions associated with a timeline and then merge/unify // with logic above to avoid Stream.concat HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); @@ -430,6 +432,13 @@ private Stream getCommitInstantsToArchive() { table.getActiveTimeline(), config.getInlineCompactDeltaCommitMax()) : Option.empty(); + // The clustering commit instant can not be archived unless we ensure that the replaced files have been cleaned, + // without the replaced files metadata on the timeline, the fs view would expose duplicates for readers. + // Meanwhile, when inline or async clustering is enabled, we need to ensure that there is a commit in the active timeline + // to check whether the file slice generated in pending clustering after archive isn't committed. + Option oldestInstantToRetainForClustering = + ClusteringUtils.getOldestInstantToRetainForClustering(table.getActiveTimeline(), table.getMetaClient()); + // Actually do the commits Stream instantToArchiveStream = commitTimeline.getInstants() .filter(s -> { @@ -442,7 +451,7 @@ private Stream getCommitInstantsToArchive() { return !(firstSavepoint.isPresent() && compareTimestamps(firstSavepoint.get().getTimestamp(), LESSER_THAN_OR_EQUALS, s.getTimestamp())); } }).filter(s -> { - // Ensure commits >= oldest pending compaction commit is retained + // Ensure commits >= the oldest pending compaction/replace commit is retained return oldestPendingCompactionAndReplaceInstant .map(instant -> compareTimestamps(instant.getTimestamp(), GREATER_THAN, s.getTimestamp())) .orElse(true); @@ -459,6 +468,10 @@ private Stream getCommitInstantsToArchive() { oldestInstantToRetainForCompaction.map(instantToRetain -> compareTimestamps(s.getTimestamp(), LESSER_THAN, instantToRetain.getTimestamp())) .orElse(true) + ).filter(s -> + oldestInstantToRetainForClustering.map(instantToRetain -> + HoodieTimeline.compareTimestamps(s.getTimestamp(), LESSER_THAN, instantToRetain.getTimestamp())) + .orElse(true) ); return instantToArchiveStream.limit(commitTimeline.countInstants() - minInstantsToKeep); } else { @@ -466,7 +479,7 @@ private Stream getCommitInstantsToArchive() { } } - private Stream getInstantsToArchive() { + private Stream getInstantsToArchive() throws IOException { Stream instants = Stream.concat(getCleanInstantsToArchive(), getCommitInstantsToArchive()); if (config.isMetastoreEnabled()) { return Stream.empty(); @@ -502,24 +515,27 @@ private Stream getInstantsToArchive() { .setBasePath(HoodieTableMetadata.getDatasetBasePath(config.getBasePath())) .setConf(metaClient.getHadoopConf()) .build(); - Option earliestActiveDatasetCommit = dataMetaClient.getActiveTimeline().firstInstant(); - - if (config.shouldArchiveBeyondSavepoint()) { - // There are chances that there could be holes in the timeline due to archival and savepoint interplay. - // So, the first non-savepoint commit in the data timeline is considered as beginning of the active timeline. - Option firstNonSavepointCommit = dataMetaClient.getActiveTimeline().getFirstNonSavepointCommit(); - if (firstNonSavepointCommit.isPresent()) { - String firstNonSavepointCommitTime = firstNonSavepointCommit.get().getTimestamp(); - instants = instants.filter(instant -> - compareTimestamps(instant.getTimestamp(), LESSER_THAN, firstNonSavepointCommitTime)); - } - } else { - // Do not archive the commits that live in data set active timeline. - // This is required by metadata table, see HoodieTableMetadataUtil#processRollbackMetadata for details. - if (earliestActiveDatasetCommit.isPresent()) { - instants = instants.filter(instant -> - compareTimestamps(instant.getTimestamp(), HoodieTimeline.LESSER_THAN, earliestActiveDatasetCommit.get().getTimestamp())); - } + Option qualifiedEarliestInstant = + TimelineUtils.getEarliestInstantForMetadataArchival( + dataMetaClient.getActiveTimeline(), config.shouldArchiveBeyondSavepoint()); + + // Do not archive the instants after the earliest commit (COMMIT, DELTA_COMMIT, and + // REPLACE_COMMIT only, considering non-savepoint commit only if enabling archive + // beyond savepoint) and the earliest inflight instant (all actions). + // This is required by metadata table, see HoodieTableMetadataUtil#processRollbackMetadata + // for details. + // Note that we cannot blindly use the earliest instant of all actions, because CLEAN and + // ROLLBACK instants are archived separately apart from commits (check + // HoodieTimelineArchiver#getCleanInstantsToArchive). If we do so, a very old completed + // CLEAN or ROLLBACK instant can block the archive of metadata table timeline and causes + // the active timeline of metadata table to be extremely long, leading to performance issues + // for loading the timeline. + if (qualifiedEarliestInstant.isPresent()) { + instants = instants.filter(instant -> + compareTimestamps( + instant.getTimestamp(), + HoodieTimeline.LESSER_THAN, + qualifiedEarliestInstant.get().getTimestamp())); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java new file mode 100644 index 0000000000000..28f308a32f907 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.exception.HoodieDeletePartitionException; +import org.apache.hudi.table.HoodieTable; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** + * A utility class for helper functions when performing a delete partition operation. + */ +public class DeletePartitionUtils { + + /** + * Check if there are any pending table service actions (requested + inflight) on a table affecting the partitions to + * be dropped. + *

+ * This check is to prevent a drop-partition from proceeding should a partition have a table service action in + * the pending stage. If this is allowed to happen, the filegroup that is an input for a table service action, might + * also be a candidate for being replaced. As such, when the table service action and drop-partition commits are + * committed, there will be two commits replacing a single filegroup. + *

+ * For example, a timeline might have an execution order as such: + * 000.replacecommit.requested (clustering filegroup_1 + filegroup_2 -> filegroup_3) + * 001.replacecommit.requested, 001.replacecommit.inflight, 0001.replacecommit (drop_partition to replace filegroup_1) + * 000.replacecommit.inflight (clustering is executed now) + * 000.replacecommit (clustering completed) + * For an execution order as shown above, 000.replacecommit and 001.replacecommit will both flag filegroup_1 to be replaced. + * This will cause downstream duplicate key errors when a map is being constructed. + * + * @param table Table to perform validation on + * @param partitionsToDrop List of partitions to drop + */ + public static void checkForPendingTableServiceActions(HoodieTable table, List partitionsToDrop) { + List instantsOfOffendingPendingTableServiceAction = new ArrayList<>(); + // ensure that there are no pending inflight clustering/compaction operations involving this partition + SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView(); + + // separating the iteration of pending compaction operations from clustering as they return different stream types + fileSystemView.getPendingCompactionOperations() + .filter(op -> partitionsToDrop.contains(op.getRight().getPartitionPath())) + .forEach(op -> instantsOfOffendingPendingTableServiceAction.add(op.getLeft())); + + fileSystemView.getFileGroupsInPendingClustering() + .filter(fgIdInstantPair -> partitionsToDrop.contains(fgIdInstantPair.getLeft().getPartitionPath())) + .forEach(x -> instantsOfOffendingPendingTableServiceAction.add(x.getRight().getTimestamp())); + + if (instantsOfOffendingPendingTableServiceAction.size() > 0) { + throw new HoodieDeletePartitionException("Failed to drop partitions. " + + "Please ensure that there are no pending table service actions (clustering/compaction) for the partitions to be deleted: " + partitionsToDrop + ". " + + "Instant(s) of offending pending table service action: " + + instantsOfOffendingPendingTableServiceAction.stream().distinct().collect(Collectors.toList())); + } + } + +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index 1180845a6ed8a..b3b6177f119bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -153,7 +153,9 @@ public class HoodieClusteringConfig extends HoodieConfig { + "RECENT_DAYS: keep a continuous range of partitions, worked together with configs '" + DAYBASED_LOOKBACK_PARTITIONS.key() + "' and '" + PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST.key() + "." + "SELECTED_PARTITIONS: keep partitions that are in the specified range ['" + PARTITION_FILTER_BEGIN_PARTITION.key() + "', '" - + PARTITION_FILTER_END_PARTITION.key() + "']."); + + PARTITION_FILTER_END_PARTITION.key() + "']." + + "DAY_ROLLING: clustering partitions on a rolling basis by the hour to avoid clustering all partitions each time, " + + "which strategy sorts the partitions asc and chooses the partition of which index is divided by 24 and the remainder is equal to the current hour."); public static final ConfigProperty PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP = ConfigProperty .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 738e2d6b48d13..855d1e4936aa4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -55,7 +55,7 @@ public class HoodieCreateHandle extends private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class); - protected final HoodieFileWriter fileWriter; + protected HoodieFileWriter fileWriter; protected final Path path; protected long recordsWritten = 0; protected long insertRecordsWritten = 0; @@ -208,7 +208,10 @@ public List close() { LOG.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten); try { - fileWriter.close(); + if (fileWriter != null) { + fileWriter.close(); + fileWriter = null; + } setupWriteStatus(); 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 9607080d1e74e..408443f0fb553 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 @@ -112,6 +112,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadataWriter.class); + public static final String METADATA_COMPACTION_TIME_SUFFIX = "001"; + // Virtual keys support for metadata table. This Field is // from the metadata payload schema. private static final String RECORD_KEY_FIELD_NAME = HoodieMetadataPayload.KEY_FIELD_NAME; @@ -242,6 +244,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi // Create the write config for the metadata table by borrowing options from the main write config. HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() + .withEngineType(writeConfig.getEngineType()) .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() .withConsistencyCheckEnabled(writeConfig.getConsistencyGuardConfig().isConsistencyCheckEnabled()) @@ -265,7 +268,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) .retainCommits(writeConfig.getMetadataCleanerCommitsRetained()) .build()) // we will trigger archive manually, to ensure only regular writer invokes it @@ -1016,22 +1019,38 @@ protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String inst // finish off any pending compactions if any from previous attempt. writeClient.runAnyPendingCompactions(); - String latestDeltaCommitTime = metadataMetaClient.reloadActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant() - .get().getTimestamp(); + String latestDeltaCommitTimeInMetadataTable = metadataMetaClient.reloadActiveTimeline() + .getDeltaCommitTimeline() + .filterCompletedInstants() + .lastInstant().orElseThrow(() -> new HoodieMetadataException("No completed deltacommit in metadata table")) + .getTimestamp(); + // we need to find if there are any inflights in data table timeline before or equal to the latest delta commit in metadata table. + // Whenever you want to change this logic, please ensure all below scenarios are considered. + // a. There could be a chance that latest delta commit in MDT is committed in MDT, but failed in DT. And so findInstantsBeforeOrEquals() should be employed + // b. There could be DT inflights after latest delta commit in MDT and we are ok with it. bcoz, the contract is, latest compaction instant time in MDT represents + // any instants before that is already synced with metadata table. + // c. Do consider out of order commits. For eg, c4 from DT could complete before c3. and we can't trigger compaction in MDT with c4 as base instant time, until every + // instant before c4 is synced with metadata table. List pendingInstants = dataMetaClient.reloadActiveTimeline().filterInflightsAndRequested() - .findInstantsBefore(instantTime).getInstants().collect(Collectors.toList()); + .findInstantsBeforeOrEquals(latestDeltaCommitTimeInMetadataTable).getInstants().collect(Collectors.toList()); if (!pendingInstants.isEmpty()) { - LOG.info(String.format("Cannot compact metadata table as there are %d inflight instants before latest deltacommit %s: %s", - pendingInstants.size(), latestDeltaCommitTime, Arrays.toString(pendingInstants.toArray()))); + LOG.info(String.format( + "Cannot compact metadata table as there are %d inflight instants in data table before latest deltacommit in metadata table: %s. Inflight instants in data table: %s", + pendingInstants.size(), latestDeltaCommitTimeInMetadataTable, Arrays.toString(pendingInstants.toArray()))); return; } // Trigger compaction with suffixes based on the same instant time. This ensures that any future // delta commits synced over will not have an instant time lesser than the last completed instant on the // metadata table. - final String compactionInstantTime = latestDeltaCommitTime + "001"; - if (writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) { + final String compactionInstantTime = latestDeltaCommitTimeInMetadataTable + METADATA_COMPACTION_TIME_SUFFIX; + // we need to avoid checking compaction w/ same instant again. + // lets say we trigger compaction after C5 in MDT and so compaction completes with C4001. but C5 crashed before completing in MDT. + // and again w/ C6, we will re-attempt compaction at which point latest delta commit is C4 in MDT. + // and so we try compaction w/ instant C4001. So, we can avoid compaction if we already have compaction w/ same instant time. + if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(compactionInstantTime) + && writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) { writeClient.compact(compactionInstantTime); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 35eb0edfbfc61..fac1db6d54526 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -470,7 +470,17 @@ public abstract Option scheduleCleaning(HoodieEngineContext c * * @return information on cleaned file slices */ - public abstract HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking); + @Deprecated + public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking) { + return clean(context, cleanInstantTime); + } + + /** + * Executes a new clean action. + * + * @return information on cleaned file slices + */ + public abstract HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime); /** * Schedule rollback for the instant time. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 64e69b1d2a9bd..737388645b4b0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java @@ -465,7 +465,24 @@ public Option getEarliestCommitToRetain() { int hoursRetained = config.getCleanerHoursRetained(); if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS && commitTimeline.countInstants() > commitsRetained) { - earliestCommitToRetain = commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained); //15 instants total, 10 commits to retain, this gives 6th instant in the list + Option earliestPendingCommits = hoodieTable.getMetaClient() + .getActiveTimeline() + .getCommitsTimeline() + .filter(s -> !s.isCompleted()).firstInstant(); + if (earliestPendingCommits.isPresent()) { + // Earliest commit to retain must not be later than the earliest pending commit + earliestCommitToRetain = + commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained).map(nthInstant -> { + if (nthInstant.compareTo(earliestPendingCommits.get()) <= 0) { + return Option.of(nthInstant); + } else { + return commitTimeline.findInstantsBefore(earliestPendingCommits.get().getTimestamp()).lastInstant(); + } + }).orElse(Option.empty()); + } else { + earliestCommitToRetain = commitTimeline.nthInstant(commitTimeline.countInstants() + - commitsRetained); //15 instants total, 10 commits to retain, this gives 6th instant in the list + } } else if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) { Instant instant = Instant.now(); ZonedDateTime currentDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java index 5d62ef390233f..8aafa6d28c425 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java @@ -71,11 +71,18 @@ public Option generateClusteringPlan() { HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient(); LOG.info("Scheduling clustering for " + metaClient.getBasePath()); HoodieWriteConfig config = getWriteConfig(); - List partitionPaths = FSUtils.getAllPartitionPaths(getEngineContext(), config.getMetadataConfig(), metaClient.getBasePath()); - // get matched partitions if set - partitionPaths = getMatchedPartitions(config, partitionPaths); - // filter the partition paths if needed to reduce list status + String partitionSelected = config.getClusteringPartitionSelected(); + List partitionPaths; + + if (StringUtils.isNullOrEmpty(partitionSelected)) { + // get matched partitions if set + partitionPaths = getRegexPatternMatchedPartitions(config, FSUtils.getAllPartitionPaths(getEngineContext(), config.getMetadataConfig(), metaClient.getBasePath())); + // filter the partition paths if needed to reduce list status + } else { + partitionPaths = Arrays.asList(partitionSelected.split(",")); + } + partitionPaths = filterPartitionPaths(partitionPaths); if (partitionPaths.isEmpty()) { @@ -114,15 +121,6 @@ public Option generateClusteringPlan() { .build()); } - public List getMatchedPartitions(HoodieWriteConfig config, List partitionPaths) { - String partitionSelected = config.getClusteringPartitionSelected(); - if (!StringUtils.isNullOrEmpty(partitionSelected)) { - return Arrays.asList(partitionSelected.split(",")); - } else { - return getRegexPatternMatchedPartitions(config, partitionPaths); - } - } - public List getRegexPatternMatchedPartitions(HoodieWriteConfig config, List partitionPaths) { String pattern = config.getClusteringPartitionFilterRegexPattern(); if (!StringUtils.isNullOrEmpty(pattern)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index 4add51886fe3a..e8acd54c5aa49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -37,6 +37,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.marker.WriteMarkersFactory; @@ -156,12 +157,13 @@ private void validateRollbackCommitSequence() { // since with LAZY rollback we support parallel writing which can allow a new inflight while rollback is ongoing // Remove this once we support LAZY rollback of failed writes by default as parallel writing becomes the default // writer mode. - if (config.getFailedWritesCleanPolicy().isEager()) { + if (config.getFailedWritesCleanPolicy().isEager() && !HoodieTableMetadata.isMetadataTable(config.getBasePath())) { final String instantTimeToRollback = instantToRollback.getTimestamp(); HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline(); // Make sure only the last n commits are being rolled back // If there is a commit in-between or after that is not rolled back, then abort + // this condition may not hold good for metadata table. since the order of commits applied to MDT is data table commits and the ordering could be different. if ((instantTimeToRollback != null) && !commitTimeline.empty() && !commitTimeline.findInstantsAfter(instantTimeToRollback, Integer.MAX_VALUE).empty()) { // check if remnants are from a previous LAZY rollback config, if yes, let out of order rollback continue diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index 7f408c1b8d24a..310ebba3c14c7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -85,15 +86,33 @@ public HoodieSavepointMetadata execute() { "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained); context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime + " " + table.getConfig().getTableName()); - List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); - Map> latestFilesMap = context.mapToPair(partitions, partitionPath -> { - // Scan all partitions files with this commit time - LOG.info("Collecting latest files in partition path " + partitionPath); - TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); - List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) - .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); - return new ImmutablePair<>(partitionPath, latestFiles); - }, null); + TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView(); + + Map> latestFilesMap; + // NOTE: for performance, we have to use different logic here for listing the latest files + // before or on the given instant: + // (1) using metadata-table-based file listing: instead of parallelizing the partition + // listing which incurs unnecessary metadata table reads, we directly read the metadata + // table once in a batch manner through the timeline server; + // (2) using direct file system listing: we parallelize the partition listing so that + // each partition can be listed on the file system concurrently through Spark. + // Note that + if (shouldUseBatchLookup(config)) { + latestFilesMap = view.getAllLatestBaseFilesBeforeOrOn(instantTime).entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().map(HoodieBaseFile::getFileName).collect(Collectors.toList()))); + } else { + List partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); + latestFilesMap = context.mapToPair(partitions, partitionPath -> { + // Scan all partitions files with this commit time + LOG.info("Collecting latest files in partition path " + partitionPath); + List latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime) + .map(HoodieBaseFile::getFileName).collect(Collectors.toList()); + return new ImmutablePair<>(partitionPath, latestFiles); + }, null); + } + HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint table.getActiveTimeline().createNewInstant( @@ -107,4 +126,22 @@ public HoodieSavepointMetadata execute() { throw new HoodieSavepointException("Failed to savepoint " + instantTime, e); } } + + /** + * Whether to use batch lookup for listing the latest base files in metadata table. + *

+ * Note that metadata table has to be enabled, and the storage type of the file system view + * cannot be EMBEDDED_KV_STORE or SPILLABLE_DISK (these two types are not integrated with + * metadata table, see HUDI-5612). + * + * @param config Write configs. + * @return {@code true} if using batch lookup; {@code false} otherwise. + */ + private boolean shouldUseBatchLookup(HoodieWriteConfig config) { + FileSystemViewStorageType storageType = + config.getClientSpecifiedViewStorageConfig().getStorageType(); + return config.getMetadataConfig().enabled() + && !FileSystemViewStorageType.EMBEDDED_KV_STORE.equals(storageType) + && !FileSystemViewStorageType.SPILLABLE_DISK.equals(storageType); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java index dfd55f2958125..625ed18227853 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/WriteMarkersFactory.java @@ -24,7 +24,6 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.HoodieTable; -import com.esotericsoftware.minlog.Log; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -47,14 +46,14 @@ public static WriteMarkers get(MarkerType markerType, HoodieTable table, String return new DirectWriteMarkers(table, instantTime); case TIMELINE_SERVER_BASED: if (!table.getConfig().isEmbeddedTimelineServerEnabled()) { - Log.warn("Timeline-server-based markers are configured as the marker type " + LOG.warn("Timeline-server-based markers are configured as the marker type " + "but embedded timeline server is not enabled. Falling back to direct markers."); return new DirectWriteMarkers(table, instantTime); } String basePath = table.getMetaClient().getBasePath(); if (StorageSchemes.HDFS.getScheme().equals( FSUtils.getFs(basePath, table.getContext().getHadoopConf().newCopy()).getScheme())) { - Log.warn("Timeline-server-based markers are not supported for HDFS: " + LOG.warn("Timeline-server-based markers are not supported for HDFS: " + "base path " + basePath + ". Falling back to direct markers."); return new DirectWriteMarkers(table, instantTime); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java new file mode 100644 index 0000000000000..929540b8dcaf7 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.SyncableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieDeletePartitionException; +import org.apache.hudi.table.HoodieTable; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class TestDeletePartitionUtils { + + private static final String PARTITION_IN_PENDING_SERVICE_ACTION = "partition_with_pending_table_service_action"; + private static final String HARDCODED_INSTANT_TIME = "0"; + + private final HoodieTable table = Mockito.mock(HoodieTable.class); + + private final SyncableFileSystemView fileSystemView = Mockito.mock(SyncableFileSystemView.class); + + public static Stream generateTruthValues() { + int noOfVariables = 2; + int noOfRows = 1 << noOfVariables; + Object[][] truthValues = new Object[noOfRows][noOfVariables]; + for (int i = 0; i < noOfRows; i++) { + for (int j = noOfVariables - 1; j >= 0; j--) { + boolean out = (i / (1 << j)) % 2 != 0; + truthValues[i][j] = out; + } + } + return Stream.of(truthValues).map(Arguments::of); + } + + @ParameterizedTest + @MethodSource("generateTruthValues") + public void testDeletePartitionUtils( + boolean hasPendingCompactionOperations, + boolean hasFileGroupsInPendingClustering) { + System.out.printf("hasPendingCompactionOperations: %s, hasFileGroupsInPendingClustering: %s%n", + hasPendingCompactionOperations, hasFileGroupsInPendingClustering); + Mockito.when(table.getSliceView()).thenReturn(fileSystemView); + Mockito.when(fileSystemView.getPendingCompactionOperations()).thenReturn(createPendingCompactionOperations(hasPendingCompactionOperations)); + Mockito.when(fileSystemView.getFileGroupsInPendingClustering()).thenReturn(createFileGroupsInPendingClustering(hasFileGroupsInPendingClustering)); + + boolean shouldThrowException = hasPendingCompactionOperations || hasFileGroupsInPendingClustering; + + if (shouldThrowException) { + assertThrows(HoodieDeletePartitionException.class, + () -> DeletePartitionUtils.checkForPendingTableServiceActions(table, + Collections.singletonList(PARTITION_IN_PENDING_SERVICE_ACTION))); + } else { + assertDoesNotThrow(() -> DeletePartitionUtils.checkForPendingTableServiceActions(table, + Collections.singletonList(PARTITION_IN_PENDING_SERVICE_ACTION))); + } + } + + private static Stream> createPendingCompactionOperations(boolean hasPendingCompactionOperations) { + return Stream.of(Pair.of(HARDCODED_INSTANT_TIME, getCompactionOperation(hasPendingCompactionOperations))); + } + + private static CompactionOperation getCompactionOperation(boolean hasPendingJobInPartition) { + return new CompactionOperation( + "fileId", getPartitionName(hasPendingJobInPartition), HARDCODED_INSTANT_TIME, Option.empty(), + new ArrayList<>(), Option.empty(), Option.empty(), new HashMap<>()); + } + + private static Stream> createFileGroupsInPendingClustering(boolean hasFileGroupsInPendingClustering) { + HoodieFileGroupId hoodieFileGroupId = new HoodieFileGroupId(getPartitionName(hasFileGroupsInPendingClustering), "fileId"); + HoodieInstant hoodieInstant = new HoodieInstant(true, "replacecommit", HARDCODED_INSTANT_TIME); + return Stream.of(Pair.of(hoodieFileGroupId, hoodieInstant)); + } + + private static String getPartitionName(boolean hasPendingTableServiceAction) { + return hasPendingTableServiceAction ? PARTITION_IN_PENDING_SERVICE_ACTION : "unaffected_partition"; + } + +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java index 440bc95615391..a053a9611050c 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java @@ -71,7 +71,7 @@ public void testFilterPartitionPaths() { fakeTimeBasedPartitionsPath.add("20210719"); fakeTimeBasedPartitionsPath.add("20210721"); - List list = strategyTestRegexPattern.getMatchedPartitions(hoodieWriteConfig, fakeTimeBasedPartitionsPath); + List list = strategyTestRegexPattern.getRegexPatternMatchedPartitions(hoodieWriteConfig, fakeTimeBasedPartitionsPath); assertEquals(2, list.size()); assertTrue(list.contains("20210721")); assertTrue(list.contains("20210723")); 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 551b412ccbc6c..9a9037c17cc07 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 @@ -69,6 +69,7 @@ import com.codahale.metrics.Timer; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,7 +101,7 @@ public class HoodieFlinkWriteClient extends * FileID to write handle mapping in order to record the write handles for each file group, * so that we can append the mini-batch data buffer incrementally. */ - private final Map> bucketToHandles; + private final Map bucketToHandles; public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) { super(context, writeConfig, FlinkUpgradeDowngradeHelper.getInstance()); @@ -126,6 +127,11 @@ protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoop return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); } + @Override + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { + return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context, metaClient); + } + @Override public List> filterExists(List> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible @@ -147,9 +153,10 @@ public List upsert(List> records, String instantTim initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime)); table.validateUpsertSchema(); preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); - final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), - instantTime, table, records.listIterator()); - HoodieWriteMetadata> result = ((HoodieFlinkTable) table).upsert(context, writeHandle, instantTime, records); + HoodieWriteMetadata> result; + try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table)) { + result = ((HoodieFlinkTable) table).upsert(context, closeableHandle.getWriteHandle(), instantTime, records); + } if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); } @@ -166,9 +173,10 @@ public List upsertPreppedRecords(List> preppedRecor Map>> preppedRecordsByFileId = preppedRecords.stream().parallel() .collect(Collectors.groupingBy(r -> r.getCurrentLocation().getFileId())); return preppedRecordsByFileId.values().stream().parallel().map(records -> { - final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), - instantTime, table, records.listIterator()); - HoodieWriteMetadata> result = ((HoodieFlinkTable) table).upsertPrepped(context, writeHandle, instantTime, records); + HoodieWriteMetadata> result; + try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table)) { + result = ((HoodieFlinkTable) table).upsertPrepped(context, closeableHandle.getWriteHandle(), instantTime, records); + } return postWrite(result, instantTime, table); }).flatMap(Collection::stream).collect(Collectors.toList()); } @@ -180,9 +188,10 @@ public List insert(List> records, String instantTim table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); // create the write handle if not exists - final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), - instantTime, table, records.listIterator()); - HoodieWriteMetadata> result = ((HoodieFlinkTable) table).insert(context, writeHandle, instantTime, records); + HoodieWriteMetadata> result; + try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table)) { + result = ((HoodieFlinkTable) table).insert(context, closeableHandle.getWriteHandle(), instantTime, records); + } if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); } @@ -203,9 +212,10 @@ public List insertOverwrite( table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient()); // create the write handle if not exists - final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), - instantTime, table, records.listIterator()); - HoodieWriteMetadata result = ((HoodieFlinkTable) table).insertOverwrite(context, writeHandle, instantTime, records); + HoodieWriteMetadata> result; + try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table)) { + result = ((HoodieFlinkTable) table).insertOverwrite(context, closeableHandle.getWriteHandle(), instantTime, records); + } return postWrite(result, instantTime, table); } @@ -222,9 +232,10 @@ public List insertOverwriteTable( table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient()); // create the write handle if not exists - final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), - instantTime, table, records.listIterator()); - HoodieWriteMetadata result = ((HoodieFlinkTable) table).insertOverwriteTable(context, writeHandle, instantTime, records); + HoodieWriteMetadata> result; + try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table)) { + result = ((HoodieFlinkTable) table).insertOverwriteTable(context, closeableHandle.getWriteHandle(), instantTime, records); + } return postWrite(result, instantTime, table); } @@ -291,10 +302,14 @@ public void initMetadataTable() { HoodieFlinkTable table = getHoodieTable(); if (config.isMetadataTableEnabled()) { // initialize the metadata table path - try (HoodieBackedTableMetadataWriter metadataWriter = initMetadataWriter()) { - // do nothing + // guard the metadata writer with concurrent lock + try { + this.txnManager.getLockManager().lock(); + initMetadataWriter().close(); } catch (Exception e) { throw new HoodieException("Failed to initialize metadata table", e); + } finally { + this.txnManager.getLockManager().unlock(); } // clean the obsolete index stats table.deleteMetadataIndexIfNecessary(); @@ -360,18 +375,21 @@ protected List postWrite(HoodieWriteMetadata> res protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, - Option> extraMetadata, - boolean acquireLockForArchival) { + Option> extraMetadata) { try { // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), createTable(config, hadoopConf), instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); - autoArchiveOnCommit(table, acquireLockForArchival); } finally { this.heartbeatClient.stop(instantTime); } } + @Override + protected void mayBeCleanAndArchive(HoodieTable table) { + autoArchiveOnCommit(table); + } + @Override public void commitCompaction( String compactionInstantTime, @@ -478,16 +496,13 @@ private void completeClustering( } @Override - protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { - // Create a Hoodie table which encapsulated the commits and files visible - return getHoodieTable(); - } - - @Override - protected void tryUpgrade(HoodieTableMetaClient metaClient, Option instantTime) { + protected void doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { // do nothing. + // flink executes the upgrade/downgrade once when initializing the first instant on start up, // no need to execute the upgrade/downgrade on each write in streaming. + + // flink performs metadata table bootstrap on the coordinator when it starts up. } public void completeTableService( @@ -526,15 +541,10 @@ public void cleanHandles() { this.bucketToHandles.clear(); } - /** - * Clean the write handles within a checkpoint interval, this operation - * would close the underneath file handles, if any error happens, clean the - * corrupted data file. - */ - public void cleanHandlesGracefully() { - this.bucketToHandles.values() - .forEach(handle -> ((MiniBatchHandle) handle).closeGracefully()); - this.bucketToHandles.clear(); + @Override + public void close() { + super.close(); + cleanHandles(); } /** @@ -558,24 +568,20 @@ public void cleanHandlesGracefully() { final String partitionPath = record.getPartitionPath(); final boolean insertClustering = config.allowDuplicateInserts(); - if (bucketToHandles.containsKey(fileID)) { - MiniBatchHandle lastHandle = (MiniBatchHandle) bucketToHandles.get(fileID); - if (lastHandle.shouldReplace()) { - HoodieWriteHandle writeHandle = insertClustering - ? new FlinkConcatAndReplaceHandle<>(config, instantTime, table, recordItr, partitionPath, fileID, - table.getTaskContextSupplier(), lastHandle.getWritePath()) - : new FlinkMergeAndReplaceHandle<>(config, instantTime, table, recordItr, partitionPath, fileID, - table.getTaskContextSupplier(), lastHandle.getWritePath()); - this.bucketToHandles.put(fileID, writeHandle); // override with new replace handle - return writeHandle; - } - } - final boolean isDelta = table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ); - final HoodieWriteHandle writeHandle; if (isDelta) { - writeHandle = new FlinkAppendHandle<>(config, instantTime, table, partitionPath, fileID, recordItr, + return new FlinkAppendHandle<>(config, instantTime, table, partitionPath, fileID, recordItr, table.getTaskContextSupplier()); + } + + final HoodieWriteHandle writeHandle; + if (bucketToHandles.containsKey(fileID)) { + Path lastPath = bucketToHandles.get(fileID); + writeHandle = insertClustering + ? new FlinkConcatAndReplaceHandle<>(config, instantTime, table, recordItr, partitionPath, fileID, + table.getTaskContextSupplier(), lastPath) + : new FlinkMergeAndReplaceHandle<>(config, instantTime, table, recordItr, partitionPath, fileID, + table.getTaskContextSupplier(), lastPath); } else if (loc.getInstantTime().equals("I")) { writeHandle = new FlinkCreateHandle<>(config, instantTime, table, partitionPath, fileID, table.getTaskContextSupplier()); @@ -586,7 +592,7 @@ public void cleanHandlesGracefully() { : new FlinkMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileID, table.getTaskContextSupplier()); } - this.bucketToHandles.put(fileID, writeHandle); + this.bucketToHandles.put(fileID, ((MiniBatchHandle) writeHandle).getWritePath()); return writeHandle; } @@ -627,4 +633,25 @@ private List getAllExistingFileIds(HoodieFlinkTable table, String par // because new commit is not complete. it is safe to mark all existing file Ids as old files return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList()); } + + private final class AutoCloseableWriteHandle implements AutoCloseable { + private final HoodieWriteHandle writeHandle; + + AutoCloseableWriteHandle( + List> records, + String instantTime, + HoodieTable>, List, List> table + ) { + this.writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(), instantTime, table, records.listIterator()); + } + + HoodieWriteHandle getWriteHandle() { + return writeHandle; + } + + @Override + public void close() { + ((MiniBatchHandle) writeHandle).closeGracefully(); + } + } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 543751a041078..6a0262321a6f0 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -319,7 +319,7 @@ public Option scheduleRollback(HoodieEngineContext context, } @Override - public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking) { + public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) { return new CleanActionExecutor(context, config, this, cleanInstantTime).execute(); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index b6951bc6b7874..83bbafc4e84fe 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -92,6 +92,11 @@ protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoop return HoodieJavaTable.create(config, context); } + @Override + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { + return HoodieJavaTable.create(config, context, metaClient); + } + @Override public List upsert(List> records, String instantTime) { @@ -197,7 +202,8 @@ protected List postWrite(HoodieWriteMetadata> res result.getWriteStats().get().size()); } - postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty(), true); + postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty()); + mayBeCleanAndArchive(hoodieTable); emitCommitMetrics(instantTime, result.getCommitMetadata().get(), hoodieTable.getMetaClient().getCommitActionType()); } @@ -228,13 +234,4 @@ protected HoodieWriteMetadata> compact(String compactionInstan public HoodieWriteMetadata> cluster(final String clusteringInstant, final boolean shouldComplete) { throw new HoodieNotSupportedException("Cluster is not supported in HoodieJavaClient"); } - - @Override - protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime, boolean initialMetadataTableIfNecessary) { - // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); - - // Create a Hoodie table which encapsulated the commits and files visible - return HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient); - } - } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index 342c018e5a269..a1646dae47356 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -217,7 +217,7 @@ public Option scheduleCleaning(HoodieEngineContext context, S @Override public HoodieCleanMetadata clean(HoodieEngineContext context, - String cleanInstantTime, boolean skipLocking) { + String cleanInstantTime) { return new CleanActionExecutor(context, config, this, cleanInstantTime).execute(); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 3c878cbc14cf8..960b6baef8eb0 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -19,7 +19,6 @@ package org.apache.hudi.table; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; @@ -47,11 +46,11 @@ public static HoodieJavaTable create(HoodieWr HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build(); - return HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient); + return HoodieJavaTable.create(config, context, metaClient); } public static HoodieJavaTable create(HoodieWriteConfig config, - HoodieJavaEngineContext context, + HoodieEngineContext context, HoodieTableMetaClient metaClient) { switch (metaClient.getTableType()) { case COPY_ON_WRITE: diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java index 1f9fcf3ef9c29..1b1754501c66f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java @@ -131,6 +131,11 @@ protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoop return HoodieSparkTable.create(config, context); } + @Override + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) { + return HoodieSparkTable.create(config, context, metaClient); + } + @Override public JavaRDD> filterExists(JavaRDD> hoodieRecords) { // Create a Hoodie table which encapsulated the commits and files visible @@ -287,7 +292,8 @@ protected JavaRDD postWrite(HoodieWriteMetadata instantTime, boolean initialMetadataTableIfNecessary) { - if (initialMetadataTableIfNecessary) { - // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation, - // if it didn't exist before - // See https://issues.apache.org/jira/browse/HUDI-3343 for more details - initializeMetadataTable(instantTime); - } - - // Create a Hoodie table which encapsulated the commits and files visible - return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); + protected void initMetadataTable(Option instantTime) { + // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation, + // if it didn't exist before + // See https://issues.apache.org/jira/browse/HUDI-3343 for more details + initializeMetadataTable(instantTime); } /** 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 272d3d479852a..81526c25bccde 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CommitUtils; @@ -132,7 +133,11 @@ protected void commit(String instantTime, Map preppedRecords = prepRecords(partitionRecordsMap); JavaRDD preppedRecordRDD = HoodieJavaRDD.getJavaRDD(preppedRecords); - try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig)) { + // rollback partially failed writes if any. + if (writeClient.rollbackFailedWrites()) { + metadataMetaClient = HoodieTableMetaClient.reload(metadataMetaClient); + } if (canTriggerTableService) { // trigger compaction before doing the delta commit. this is to ensure, if this delta commit succeeds in metadata table, but failed in data table, // we would have compacted metadata table and so could have included uncommitted data which will never be ignored while reading from metadata @@ -162,7 +167,7 @@ protected void commit(String instantTime, Map statuses = writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect(); statuses.forEach(writeStatus -> { if (writeStatus.hasErrors()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 743aff51a1254..fc159f5ea8e38 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -265,8 +265,8 @@ public Iterator> handleInsert( } @Override - public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking) { - return new CleanActionExecutor<>(context, config, this, cleanInstantTime, skipLocking).execute(); + public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) { + return new CleanActionExecutor<>(context, config, this, cleanInstantTime, false).execute(); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 66d51c91283f3..3719ed742bcb3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -19,7 +19,6 @@ package org.apache.hudi.table; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; @@ -60,11 +59,11 @@ public static HoodieSparkTable create(HoodieW .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setProperties(config.getProps()).build(); - return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient); + return HoodieSparkTable.create(config, context, metaClient); } public static HoodieSparkTable create(HoodieWriteConfig config, - HoodieSparkEngineContext context, + HoodieEngineContext context, HoodieTableMetaClient metaClient) { HoodieSparkTable hoodieSparkTable; switch (metaClient.getTableType()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index f8e4b31ff687e..06127b8aa4fbf 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -379,8 +379,7 @@ protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, } @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { + public Iterator> handleInsert(String idPfx, Iterator> recordItr) { // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records if (!recordItr.hasNext()) { LOG.info("Empty partition"); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java index 65a45e1c6a047..f4572bad6db88 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBucketIndexPartitioner.java @@ -27,6 +27,7 @@ import java.util.Map.Entry; import java.util.Set; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.index.bucket.BucketIdentifier; import scala.Tuple2; @@ -43,6 +44,9 @@ import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.WorkloadStat; +import static org.apache.hudi.common.model.WriteOperationType.INSERT_OVERWRITE; +import static org.apache.hudi.common.model.WriteOperationType.INSERT_OVERWRITE_TABLE; + /** * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). */ @@ -58,6 +62,7 @@ public class SparkBucketIndexPartitioner> exten * The partition offset is a multiple of the bucket num. */ private final Map partitionPathOffset; + private final boolean isOverwrite; /** * Partition path and file groups in it pair. Decide the file group an incoming update should go to. @@ -85,6 +90,8 @@ public SparkBucketIndexPartitioner(WorkloadProfile profile, i += numBuckets; } assignUpdates(profile); + WriteOperationType operationType = profile.getOperationType(); + this.isOverwrite = INSERT_OVERWRITE.equals(operationType) || INSERT_OVERWRITE_TABLE.equals(operationType); } private void assignUpdates(WorkloadProfile profile) { @@ -107,6 +114,10 @@ private void assignUpdates(WorkloadProfile profile) { public BucketInfo getBucketInfo(int bucketNumber) { String partitionPath = partitionPaths.get(bucketNumber / numBuckets); String bucketId = BucketIdentifier.bucketIdStr(bucketNumber % numBuckets); + // Insert overwrite always generates new bucket file id + if (isOverwrite) { + return new BucketInfo(BucketType.INSERT, BucketIdentifier.newBucketFileIdPrefix(bucketId), partitionPath); + } Option fileIdOption = Option.fromJavaOptional(updatePartitionPathFileIds .getOrDefault(partitionPath, Collections.emptySet()).stream() .filter(e -> e.startsWith(bucketId)) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java index 149aef03e238a..8923b47fb8757 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.utils.DeletePartitionUtils; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -58,6 +59,8 @@ public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { + DeletePartitionUtils.checkForPendingTableServiceActions(table, partitions); + try { HoodieTimer timer = new HoodieTimer().startTimer(); context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions."); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 518063ed34186..380470fc25524 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -35,6 +35,7 @@ import org.apache.spark.Partitioner; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -86,4 +87,17 @@ protected List getAllExistingFileIds(String partitionPath) { // because new commit is not complete. it is safe to mark all existing file Ids as old files return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList()); } + + @Override + protected Iterator> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr, Partitioner partitioner) { + SparkHoodiePartitioner upsertPartitioner = (SparkHoodiePartitioner) partitioner; + BucketInfo binfo = upsertPartitioner.getBucketInfo(partition); + BucketType btype = binfo.bucketType; + switch (btype) { + case INSERT: + return handleInsert(binfo.fileIdPrefix, recordItr); + default: + throw new AssertionError("Expect INSERT bucketType for insert overwrite, please correct the logical of " + partitioner.getClass().getName()); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java index dd545d5262846..056b3beec842b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; @@ -41,6 +42,20 @@ public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineCont super(profile, context, table, config); } + @Override + public BucketInfo getBucketInfo(int bucketNumber) { + BucketInfo bucketInfo = super.getBucketInfo(bucketNumber); + switch (bucketInfo.bucketType) { + case INSERT: + return bucketInfo; + case UPDATE: + // Insert overwrite always generates new bucket file id + return new BucketInfo(BucketType.INSERT, FSUtils.createNewFileIdPfx(), bucketInfo.partitionPath); + default: + throw new AssertionError(); + } + } + /** * Returns a list of small files in the given partition path. */ diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java index 61e6f25af9429..fe467ddbffbeb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java @@ -87,8 +87,7 @@ public Iterator> handleUpdate(String partitionPath, String fil } @Override - public Iterator> handleInsert(String idPfx, Iterator> recordItr) - throws Exception { + public Iterator> handleInsert(String idPfx, Iterator> recordItr) { // If canIndexLogFiles, write inserts to log files else write inserts to base files if (table.getIndex().canIndexLogFiles()) { return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java new file mode 100644 index 0000000000000..7c578ad3cdee0 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.avro.model.HoodieSavepointPartitionMetadata; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +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.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieStorageConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.testutils.MetadataMergeWriteStatus; + +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.table.view.FileSystemViewStorageType.EMBEDDED_KV_STORE; +import static org.apache.hudi.common.table.view.FileSystemViewStorageType.MEMORY; +import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME; +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases for savepoint operation. + */ +public class TestSavepoint extends HoodieClientTestBase { + + private static Stream testSavepointParams() { + return Arrays.stream(new Object[][] { + {true, MEMORY}, {true, EMBEDDED_KV_STORE}, + {false, MEMORY}, {false, EMBEDDED_KV_STORE} + }).map(Arguments::of); + } + + @ParameterizedTest + @MethodSource("testSavepointParams") + public void testSavepoint(boolean enableMetadataTable, + FileSystemViewStorageType storageType) throws IOException { + HoodieWriteConfig cfg = getWriteConfig(enableMetadataTable, storageType); + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0x17AB); + try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) { + String newCommitTime = "001"; + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 200); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.upsert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + client.savepoint("user", "hoodie-savepoint-unit-test"); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); + HoodieTimeline savepointTimeline = table.getActiveTimeline().getSavePointTimeline(); + assertEquals(1, savepointTimeline.countInstants()); + + Map savepointPartitionMetadataMap = + TimelineMetadataUtils.deserializeHoodieSavepointMetadata( + savepointTimeline.getInstantDetails(savepointTimeline.firstInstant().get()).get()) + .getPartitionMetadata(); + + HoodieTimeline commitsTimeline = table.getActiveTimeline().getCommitsTimeline(); + Map> partitionToWriteStats = HoodieCommitMetadata.fromBytes( + commitsTimeline.getInstantDetails(commitsTimeline.firstInstant().get()).get(), + HoodieCommitMetadata.class) + .getPartitionToWriteStats(); + + assertEquals(partitionToWriteStats.size(), savepointPartitionMetadataMap.size()); + for (Map.Entry> entry : partitionToWriteStats.entrySet()) { + String partition = entry.getKey(); + assertTrue(savepointPartitionMetadataMap.containsKey(partition)); + assertEquals( + entry.getValue().stream().map(path -> getFileNameFromPath(path.getPath())) + .sorted().collect(Collectors.toList()), + savepointPartitionMetadataMap.get(partition).getSavepointDataFile() + .stream().sorted().collect(Collectors.toList()) + ); + } + } + } + + private HoodieWriteConfig getWriteConfig(boolean enableMetadataTable, + FileSystemViewStorageType storageType) { + return HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withBulkInsertParallelism(2) + .withFinalizeWriteParallelism(2) + .withDeleteParallelism(2) + .withWriteStatusClass(MetadataMergeWriteStatus.class) + .withConsistencyGuardConfig( + ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build()) + .withCompactionConfig( + HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build()) + .withStorageConfig(HoodieStorageConfig.newBuilder() + .hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024) + .orcMaxFileSize(1024 * 1024).build()) + .forTable(RAW_TRIPS_TEST_NAME) + .withEmbeddedTimelineServerEnabled(true) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server + .withRemoteServerPort(timelineServicePort) + .withStorageType(storageType) + .build()) + .withMetadataConfig( + HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) + .build(); + } + + private String getFileNameFromPath(String path) { + String[] parts = path.split("/"); + return parts[parts.length - 1]; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 8ea6c2adf895f..8f4da37fbda6f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -152,6 +152,7 @@ import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.metadata.HoodieBackedTableMetadataWriter.METADATA_COMPACTION_TIME_SUFFIX; import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; import static org.apache.hudi.metadata.MetadataPartitionType.FILES; @@ -580,8 +581,7 @@ public void testUpdationOfPopulateMetaFieldsForMetadataTable() throws Exception @Test public void testMetadataInsertUpsertCleanNonPartitioned() throws Exception { - HoodieTableType tableType = COPY_ON_WRITE; - init(tableType); + init(COPY_ON_WRITE); doWriteOperationNonPartitioned(testTable, "0000001", INSERT); doWriteOperationNonPartitioned(testTable, "0000002", UPSERT); testTable.doCleanBasedOnCommits("0000003", Arrays.asList("0000001")); @@ -609,8 +609,7 @@ public void testInsertUpsertCluster(HoodieTableType tableType) throws Exception */ @Test public void testMetadataTableServices() throws Exception { - HoodieTableType tableType = COPY_ON_WRITE; - init(tableType, false); + init(COPY_ON_WRITE, false); writeConfig = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(true) @@ -735,8 +734,7 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws I @ParameterizedTest @ValueSource(booleans = {true, false}) public void testVirtualKeysInBaseFiles(boolean populateMetaFields) throws Exception { - HoodieTableType tableType = MERGE_ON_READ; - init(tableType, false); + init(MERGE_ON_READ, false); writeConfig = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(true) @@ -776,14 +774,61 @@ public void testVirtualKeysInBaseFiles(boolean populateMetaFields) throws Except }); } + @Test + public void testMetadataTableCompactionWithPendingInstants() throws Exception { + init(COPY_ON_WRITE, false); + writeConfig = getWriteConfigBuilder(true, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .enableFullScan(true) + .enableMetrics(false) + .withMaxNumDeltaCommitsBeforeCompaction(4) + .build()).build(); + initWriteConfigAndMetatableWriter(writeConfig, true); + doWriteOperation(testTable, "0000001", INSERT); + String commitInstant = "0000002"; + doWriteOperation(testTable, commitInstant, INSERT); + + // test multi-writer scenario. lets add 1,2,3,4 where 1,2,4 succeeded, but 3 is still inflight. so latest delta commit in MDT is 4, while 3 is still pending + // in DT and not seen by MDT yet. compaction should not trigger until 3 goes to completion. + + // create an inflight commit for 3 + HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("0000003", UPSERT, emptyList(), + asList("p1", "p2"), 2, false, true); + doWriteOperation(testTable, "0000004"); + HoodieTableMetadata tableMetadata = metadata(writeConfig, context); + // verify that compaction of metadata table does not kick in. + assertFalse(tableMetadata.getLatestCompactionTime().isPresent()); + doWriteOperation(testTable, "0000005", INSERT); + doWriteOperation(testTable, "0000006", INSERT); + doWriteOperation(testTable, "0000007", INSERT); + + tableMetadata = metadata(writeConfig, context); + // verify that compaction of metadata table does not kick in. + assertFalse(tableMetadata.getLatestCompactionTime().isPresent()); + + // move inflight to completed + testTable.moveInflightCommitToComplete("0000003", inflightCommitMeta); + + // we have to add another commit for compaction to trigger. if not, latest delta commit in MDT is 7, but the new incoming i.e 3 is still inflight in DT while "3" + // is getting applied to MDT. + doWriteOperation(testTable, "0000008", INSERT); + // verify compaction kicked in now + String metadataCompactionInstant = "0000007" + METADATA_COMPACTION_TIME_SUFFIX; + tableMetadata = metadata(writeConfig, context); + assertTrue(tableMetadata.getLatestCompactionTime().isPresent()); + assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant); + // do full metadata validation + validateMetadata(testTable, true); + } + /** * Tests that virtual key configs are honored in base files after compaction in metadata table. */ @ParameterizedTest @ValueSource(booleans = {true, false}) public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompaction) throws Exception { - HoodieTableType tableType = COPY_ON_WRITE; - init(tableType, false); + init(COPY_ON_WRITE, false); writeConfig = getWriteConfigBuilder(true, true, false) .withMetadataConfig(HoodieMetadataConfig.newBuilder() .enable(true) @@ -802,7 +847,7 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact doWriteOperation(testTable, "0000003", INSERT); HoodieTableMetadata tableMetadata = metadata(writeConfig, context); - String metadataCompactionInstant = commitInstant + "001"; + String metadataCompactionInstant = commitInstant + METADATA_COMPACTION_TIME_SUFFIX; assertTrue(tableMetadata.getLatestCompactionTime().isPresent()); assertEquals(tableMetadata.getLatestCompactionTime().get(), metadataCompactionInstant); @@ -860,6 +905,65 @@ public void testMetadataTableWithPendingCompaction(boolean simulateFailedCompact } } + @Test + public void testMetadataRollbackWithCompaction() throws Exception { + HoodieTableType tableType = COPY_ON_WRITE; + init(tableType, false); + writeConfig = getWriteConfigBuilder(false, true, false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder() + .enable(true) + .withPopulateMetaFields(true) + .build()) + .build(); + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + + try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { + // Write 1 (Bulk insert) + String newCommitTime1 = "0000001"; + List records = dataGen.generateInserts(newCommitTime1, 100); + client.startCommitWithTime(newCommitTime1); + JavaRDD writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime1); + client.commit(newCommitTime1, writeStatuses); + + String newCommitTime2 = "0000002"; + records = dataGen.generateUniqueUpdates(newCommitTime2, 20); + client.startCommitWithTime(newCommitTime2); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime2); + client.commit(newCommitTime2, writeStatuses); + + String newCommitTime3 = "0000003"; + records = dataGen.generateUniqueUpdates(newCommitTime3, 20); + client.startCommitWithTime(newCommitTime3); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime3); + client.commit(newCommitTime3, writeStatuses); + + // rollback "3" so that there is no "3" in data table timeline, but there exists a DC "3" in metadata timeline. + client.rollback(newCommitTime3); + + // mimicing crash or making an inflight in metadata table. + Path toDelete = new Path(metaClient.getMetaPath() + "/metadata/.hoodie/" + newCommitTime2 + "." + HoodieTimeline.DELTA_COMMIT_ACTION); + metaClient.getFs().delete(toDelete); + + // re-ingest w/ same commit time. + records = dataGen.generateUniqueUpdates(newCommitTime3, 20); + client.startCommitWithTime(newCommitTime3); + writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime3); + client.commit(newCommitTime3, writeStatuses); + + // collect all commit meta files from metadata table. + FileStatus[] metaFiles = metaClient.getFs().listStatus(new Path(metaClient.getMetaPath() + "/metadata/.hoodie")); + List commit3Files = Arrays.stream(metaFiles).filter(fileStatus -> + fileStatus.getPath().getName().equals(newCommitTime3 + "." + HoodieTimeline.DELTA_COMMIT_ACTION)).collect(Collectors.toList()); + List rollbackFiles = Arrays.stream(metaFiles).filter(fileStatus -> + fileStatus.getPath().getName().endsWith("." + HoodieTimeline.ROLLBACK_ACTION)).collect(Collectors.toList()); + + // ensure commit2's delta commit in MDT has last mod time > the actual rollback for previous failed commit i.e. commit2. + // if rollback wasn't eager, rollback's last mod time will be lower than the commit3'd delta commit last mod time. + assertTrue(commit3Files.get(0).getModificationTime() > rollbackFiles.get(0).getModificationTime()); + } + } + /** * Test arguments - Table type, populate meta fields, exclude key from payload. */ @@ -1163,8 +1267,7 @@ public void testRollbackOperations(HoodieTableType tableType) throws Exception { @Test public void testRollbackOperationsNonPartitioned() throws Exception { - HoodieTableType tableType = COPY_ON_WRITE; - init(tableType); + init(COPY_ON_WRITE); doWriteInsertAndUpsertNonPartitioned(testTable); // trigger an upsert @@ -1189,8 +1292,7 @@ public void testRollbackOperationsNonPartitioned() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testManualRollbacks(final boolean populateMateFields) throws Exception { - HoodieTableType tableType = COPY_ON_WRITE; - init(tableType, false); + init(COPY_ON_WRITE, false); // Setting to archive more aggressively on the Metadata Table than the Dataset final int maxDeltaCommitsBeforeCompaction = 4; final int minArchiveCommitsMetadata = 2; @@ -1555,6 +1657,54 @@ public void testColStatsPrefixLookup() throws IOException { } } + @Test + public void testEagerRollbackinMDT() throws IOException { + tableType = MERGE_ON_READ; + initPath(); + init(tableType); + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig); + // Write 1 (Bulk insert) + String commit1 = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(commit1, 20); + client.startCommitWithTime(commit1); + List writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), commit1).collect(); + assertNoWriteErrors(writeStatuses); + + // Write 2 (inserts) + String commit2 = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(commit2); + records = dataGen.generateInserts(commit2, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), commit2).collect(); + assertNoWriteErrors(writeStatuses); + // remove latest completed delta commit from MDT. + Path toDelete = new Path(metaClient.getMetaPath() + "/metadata/.hoodie/" + commit2 + "." + HoodieTimeline.DELTA_COMMIT_ACTION); + metaClient.getFs().delete(toDelete); + + // Write 3 (updates) + client = new SparkRDDWriteClient(engineContext, writeConfig); + String commit3 = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(commit3); + records = dataGen.generateUniqueUpdates(commit3, 10); + writeStatuses = client.upsert(jsc.parallelize(records, 1), commit3).collect(); + assertNoWriteErrors(writeStatuses); + + // ensure that 000003 is after rollback of the partially failed 2nd commit. + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setBasePath(metaClient.getMetaPath() + "/metadata/").setConf(metaClient.getHadoopConf()).build(); + HoodieInstant rollbackInstant = metadataMetaClient.getActiveTimeline().getRollbackTimeline().getInstants().collect(Collectors.toList()).get(0); + + // collect all commit meta files from metadata table. + FileStatus[] metaFiles = metaClient.getFs().listStatus(new Path(metaClient.getMetaPath() + "/metadata/.hoodie")); + List commit3Files = Arrays.stream(metaFiles).filter(fileStatus -> + fileStatus.getPath().getName().equals(commit3 + "." + HoodieTimeline.DELTA_COMMIT_ACTION)).collect(Collectors.toList()); + List rollbackFiles = Arrays.stream(metaFiles).filter(fileStatus -> + fileStatus.getPath().getName().equals(rollbackInstant.getTimestamp() + "." + HoodieTimeline.ROLLBACK_ACTION)).collect(Collectors.toList()); + + // ensure commit3's delta commit in MDT has last mod time > the actual rollback for previous failed commit i.e. commit2. + // if rollback wasn't eager, rollback's last mod time will be lower than the commit3'd delta commit last mod time. + assertTrue(commit3Files.get(0).getModificationTime() > rollbackFiles.get(0).getModificationTime()); + } + /** * Test all major table operations with the given table, config and context. * @@ -1818,6 +1968,69 @@ public void testReattemptOfFailedClusteringCommit() throws Exception { validateMetadata(client); } + /** + * Validates that if an instant is completed in MDT, but crashed before commiting to DT, MDT compaction should not kick in based on the instant time + * since its not complete in DT yet. + * @throws Exception + */ + @Test + public void testMDTCompactionWithFailedCommits() throws Exception { + tableType = HoodieTableType.COPY_ON_WRITE; + init(tableType); + context = new HoodieSparkEngineContext(jsc); + HoodieWriteConfig initialConfig = getSmallInsertWriteConfig(2000, TRIP_EXAMPLE_SCHEMA, 10, false); + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withProperties(initialConfig.getProps()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(4).build()).build(); + SparkRDDWriteClient client = getHoodieWriteClient(config); + + // Write 1 (Bulk insert) + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + List records = dataGen.generateInserts(newCommitTime, 20); + client.startCommitWithTime(newCommitTime); + List writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // Write 2 (inserts) + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client); + + // setup clustering config. + HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10) + .withClusteringSortColumns("_row_key").withInlineClustering(true) + .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build(); + + HoodieWriteConfig newWriteConfig = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.BLOOM, HoodieFailedWritesCleaningPolicy.EAGER) + .withAutoCommit(false) + .withClusteringConfig(clusteringConfig).build(); + + // trigger clustering + SparkRDDWriteClient newClient = getHoodieWriteClient(newWriteConfig); + String clusteringCommitTime = newClient.scheduleClustering(Option.empty()).get().toString(); + HoodieWriteMetadata> clusterMetadata = newClient.cluster(clusteringCommitTime, true); + + // manually remove clustering completed instant from .hoodie folder and to mimic succeeded clustering in metadata table, but failed in data table. + FileCreateUtils.deleteReplaceCommit(basePath, clusteringCommitTime); + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieWriteConfig updatedWriteConfig = HoodieWriteConfig.newBuilder().withProperties(initialConfig.getProps()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(4).build()) + .withRollbackUsingMarkers(false).build(); + + client = getHoodieWriteClient(updatedWriteConfig); + + newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + records = dataGen.generateInserts(newCommitTime, 20); + writeStatuses = client.insert(jsc.parallelize(records, 1), newCommitTime).collect(); + assertNoWriteErrors(writeStatuses); + validateMetadata(client, Option.of(clusteringCommitTime)); + } + @Test public void testMetadataReadWithNoCompletedCommits() throws Exception { init(HoodieTableType.COPY_ON_WRITE); @@ -2384,6 +2597,10 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, HoodieIndex. } private void validateMetadata(SparkRDDWriteClient testClient) throws IOException { + validateMetadata(testClient, Option.empty()); + } + + private void validateMetadata(SparkRDDWriteClient testClient, Option ignoreFilesWithCommit) throws IOException { HoodieWriteConfig config = testClient.getConfig(); SparkRDDWriteClient client; @@ -2432,7 +2649,12 @@ private void validateMetadata(SparkRDDWriteClient testClient) throws IOException } else { partitionPath = new Path(basePath, partition); } + FileStatus[] fsStatuses = FSUtils.getAllDataFilesInPartition(fs, partitionPath); + if (ignoreFilesWithCommit.isPresent()) { + fsStatuses = Arrays.stream(fsStatuses).filter(fileStatus -> !fileStatus.getPath().getName().contains(ignoreFilesWithCommit.get())) + .collect(Collectors.toList()).toArray(new FileStatus[0]); + } FileStatus[] metaStatuses = tableMetadata.getAllFilesInPartition(partitionPath); List fsFileNames = Arrays.stream(fsStatuses) .map(s -> s.getPath().getName()).collect(Collectors.toList()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 081b717146a6e..2682a74142e73 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -28,9 +28,11 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.clustering.plan.strategy.SparkSingleFileSortPlanStrategy; import org.apache.hudi.client.clustering.run.strategy.SparkSingleFileSortExecutionStrategy; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.client.validator.SparkPreCommitValidator; import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator; import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator; +import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -48,6 +50,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.marker.MarkerType; @@ -77,6 +80,7 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -2409,9 +2413,13 @@ public void testRollbackAfterConsistencyCheckFailureUsingMarkers(boolean enableO testRollbackAfterConsistencyCheckFailureUsingFileList(true, enableOptimisticConsistencyGuard, populateMetCols); } - @ParameterizedTest - @MethodSource("rollbackFailedCommitsParams") - public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) throws Exception { + //@ParameterizedTest + //@MethodSource("rollbackFailedCommitsParams") + @Test + public void testRollbackFailedCommits() throws Exception { + // HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields + HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.NEVER; + boolean populateMetaFields = true; HoodieTestUtils.init(hadoopConf, basePath); SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); @@ -2470,11 +2478,12 @@ public void testRollbackFailedCommits(HoodieFailedWritesCleaningPolicy cleaningP == 0); assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 3); } else if (cleaningPolicy.isNever()) { + // never will get translated to Lazy if OCC is enabled. assertTrue( timeline .getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION)) .countInstants() - == 0); + == 2); // There should be no clean or rollback action on the timeline assertTrue( timeline @@ -2540,8 +2549,9 @@ public void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMetaFi client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); client.startCommit(); timeline = metaClient.getActiveTimeline().reload(); + // since OCC is enabled, hudi auto flips the cleaningPolicy to Lazy. assertTrue(timeline.getTimelineOfActions( - CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 5); + CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants() == 3); assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 1); } @@ -2774,6 +2784,14 @@ protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient meta } private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleaningPolicy cleaningPolicy, boolean populateMetaFields) { + Properties properties = new Properties(); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "3000"); + properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "20"); + if (!populateMetaFields) { + getPropertiesForKeyGen(populateMetaFields).entrySet().forEach(kv -> + properties.put(kv.getKey(), kv.getValue())); + } return getConfigBuilder() .withEmbeddedTimelineServerEnabled(false) .withCleanConfig(HoodieCleanConfig.newBuilder() @@ -2784,7 +2802,11 @@ private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleani .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) .withAutoCommit(false) - .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(InProcessLockProvider.class) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withProperties(properties).build(); } public static class FailingPreCommitValidator> extends SparkPreCommitValidator { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java new file mode 100644 index 0000000000000..63d959f8643af --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.functional; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.view.FileSystemViewManager; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieBackedTestDelayedTableMetadata; +import org.apache.hudi.metadata.HoodieMetadataFileSystemView; +import org.apache.hudi.testutils.HoodieClientTestHarness; +import org.apache.hudi.timeline.service.TimelineService; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.view.FileSystemViewStorageConfig.REMOTE_PORT_NUM; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests the {@link RemoteHoodieTableFileSystemView} with metadata table enabled, using + * {@link HoodieMetadataFileSystemView} on the timeline server. + */ +public class TestRemoteFileSystemViewWithMetadataTable extends HoodieClientTestHarness { + private static final Logger LOG = LogManager.getLogger(TestRemoteFileSystemViewWithMetadataTable.class); + + @BeforeEach + public void setUp() throws Exception { + initPath(); + initSparkContexts(); + initFileSystem(); + initMetaClient(); + initTimelineService(); + dataGen = new HoodieTestDataGenerator(0x1f86); + } + + @AfterEach + public void tearDown() throws Exception { + cleanupTimelineService(); + cleanupClients(); + cleanupSparkContexts(); + cleanupFileSystem(); + cleanupExecutorService(); + dataGen = null; + System.gc(); + } + + @Override + public void initTimelineService() { + // Start a timeline server that are running across multiple commits + HoodieLocalEngineContext localEngineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + + try { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(incrementTimelineServicePortToUse()).build()) + .build(); + timelineService = new TimelineService(localEngineContext, new Configuration(), + TimelineService.Config.builder().enableMarkerRequests(true) + .serverPort(config.getViewStorageConfig().getRemoteViewServerPort()).build(), + FileSystem.get(new Configuration()), + FileSystemViewManager.createViewManager( + context, config.getMetadataConfig(), config.getViewStorageConfig(), + config.getCommonConfig(), + () -> new HoodieBackedTestDelayedTableMetadata( + context, config.getMetadataConfig(), basePath, + config.getViewStorageConfig().getSpillableDir(), true))); + timelineService.startService(); + timelineServicePort = timelineService.getServerPort(); + LOG.info("Started timeline server on port: " + timelineServicePort); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testMORGetLatestFileSliceWithMetadataTable(boolean useExistingTimelineServer) throws IOException { + // This test utilizes the `HoodieBackedTestDelayedTableMetadata` to make sure the + // synced file system view is always served. + + SparkRDDWriteClient writeClient = createWriteClient( + useExistingTimelineServer ? Option.of(timelineService) : Option.empty()); + + for (int i = 0; i < 3; i++) { + writeToTable(i, writeClient); + } + + // At this point, there are three deltacommits and one compaction commit in the Hudi timeline, + // and the file system view of timeline server is not yet synced + HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.builder() + .setConf(metaClient.getHadoopConf()) + .setBasePath(basePath) + .build(); + HoodieActiveTimeline timeline = newMetaClient.getActiveTimeline(); + HoodieInstant compactionCommit = timeline.lastInstant().get(); + assertTrue(timeline.lastInstant().get().getAction().equals(COMMIT_ACTION)); + + // For all the file groups compacted by the compaction commit, the file system view + // should return the latest file slices which is written by the latest commit + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + timeline.getInstantDetails(compactionCommit).get(), HoodieCommitMetadata.class); + List> partitionFileIdPairList = + commitMetadata.getPartitionToWriteStats().entrySet().stream().flatMap( + entry -> { + String partitionPath = entry.getKey(); + return entry.getValue().stream().map( + writeStat -> Pair.of(partitionPath, writeStat.getFileId())); + } + ).collect(Collectors.toList()); + List> lookupList = new ArrayList<>(); + // Accumulate enough threads to test read-write concurrency while syncing the file system + // view at the timeline server + while (lookupList.size() < 128) { + lookupList.addAll(partitionFileIdPairList); + } + + int timelineServerPort = useExistingTimelineServer + ? timelineService.getServerPort() + : writeClient.getTimelineServer().get().getRemoteFileSystemViewConfig().getRemoteViewServerPort(); + + LOG.info("Connecting to Timeline Server: " + timelineServerPort); + RemoteHoodieTableFileSystemView view = + new RemoteHoodieTableFileSystemView("localhost", timelineServerPort, metaClient); + + List callableList = lookupList.stream() + .map(pair -> new TestViewLookUpCallable(view, pair, compactionCommit.getTimestamp())) + .collect(Collectors.toList()); + List> resultList = new ArrayList<>(); + + ExecutorService pool = Executors.newCachedThreadPool(); + callableList.forEach(callable -> { + resultList.add(pool.submit(callable)); + }); + + assertTrue(resultList.stream().map(future -> { + try { + return future.get(); + } catch (Exception e) { + LOG.error(e); + return false; + } + }).reduce((a, b) -> a && b).get()); + } + + @Override + protected HoodieTableType getTableType() { + return HoodieTableType.MERGE_ON_READ; + } + + private SparkRDDWriteClient createWriteClient(Option timelineService) { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA) + .withParallelism(2, 2) + .withBulkInsertParallelism(2) + .withFinalizeWriteParallelism(2) + .withDeleteParallelism(2) + .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) + .withMergeSmallFileGroupCandidatesLimit(0) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(3) + .build()) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withStorageType(FileSystemViewStorageType.REMOTE_ONLY) + .withRemoteServerPort(timelineService.isPresent() + ? timelineService.get().getServerPort() : REMOTE_PORT_NUM.defaultValue()) + .build()) + .withAutoCommit(false) + .forTable("test_mor_table") + .build(); + return new SparkRDDWriteClient(context, writeConfig, timelineService); + } + + private void writeToTable(int round, SparkRDDWriteClient writeClient) throws IOException { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + writeClient.startCommitWithTime(instantTime); + List records = round == 0 + ? dataGen.generateInserts(instantTime, 100) + : dataGen.generateUpdates(instantTime, 100); + + JavaRDD writeStatusRDD = writeClient.upsert(jsc.parallelize(records, 1), instantTime); + writeClient.commit(instantTime, writeStatusRDD, Option.empty(), DELTA_COMMIT_ACTION, Collections.emptyMap()); + // Triggers compaction + writeClient.scheduleCompaction(Option.empty()); + writeClient.runAnyPendingCompactions(); + } + + /** + * Test callable to send lookup request to the timeline server for the latest file slice + * based on the partition path and file ID. + */ + class TestViewLookUpCallable implements Callable { + private final RemoteHoodieTableFileSystemView view; + private final Pair partitionFileIdPair; + private final String expectedCommitTime; + + public TestViewLookUpCallable( + RemoteHoodieTableFileSystemView view, + Pair partitionFileIdPair, + String expectedCommitTime) { + this.view = view; + this.partitionFileIdPair = partitionFileIdPair; + this.expectedCommitTime = expectedCommitTime; + } + + @Override + public Boolean call() throws Exception { + Option latestFileSlice = view.getLatestFileSlice( + partitionFileIdPair.getLeft(), partitionFileIdPair.getRight()); + boolean result = latestFileSlice.isPresent() && expectedCommitTime.equals( + FSUtils.getCommitTime(new Path(latestFileSlice.get().getBaseFile().get().getPath()).getName())); + if (!result) { + LOG.error("The timeline server does not return the correct result: latestFileSliceReturned=" + + latestFileSlice + " expectedCommitTime=" + expectedCommitTime); + } + return result; + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java index 1c6973db746bc..5477cda59049b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java @@ -68,7 +68,7 @@ public void testFileGroupLookUpManyEntriesWithSameStartValue() { updateExpectedMatchesToTest(toInsert); keyRangeLookupTree.insert(toInsert); for (int i = 0; i < 10; i++) { - endKey += 1 + RANDOM.nextInt(100); + endKey += 1 + RANDOM.nextInt(50); toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString()); updateExpectedMatchesToTest(toInsert); keyRangeLookupTree.insert(toInsert); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index bc7de2f175a44..7c3e35258c783 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -807,7 +807,12 @@ public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) List originalCommits = commitsList.getKey(); List commitsAfterArchival = commitsList.getValue(); if (enableMetadata) { - assertEquals(originalCommits, commitsAfterArchival); + if (i != 6) { + assertEquals(originalCommits, commitsAfterArchival); + } else { + // on 6th commit, archival will kick in. but will archive only one commit since 2nd compaction commit is inflight. + assertEquals(originalCommits.size() - commitsAfterArchival.size(), 1); + } } else { if (i != 6) { assertEquals(originalCommits, commitsAfterArchival); @@ -1301,8 +1306,15 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { .setLoadActiveTimelineOnLoad(true).build(); for (int i = 1; i <= 17; i++) { - testTable.doWriteOperation("000000" + String.format("%02d", i), WriteOperationType.UPSERT, - i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + if (i != 2) { + testTable.doWriteOperation("000000" + String.format("%02d", i), WriteOperationType.UPSERT, + i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + } else { + // For i == 2, roll back the first commit "00000001", so the active timeline of the + // data table has one rollback instant + // The completed rollback should not block the archival in the metadata table + testTable.doRollback("00000001", "00000002"); + } // archival archiveAndGetCommitsList(writeConfig); @@ -1324,10 +1336,9 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { } else if (i == 8) { // i == 8 // The instant "00000000000000" was archived since it's less than - // the earliest instant on the dataset active timeline, - // the dataset active timeline has instants of range [00000001 ~ 00000008] - // because when it does the archiving, no compaction instant on the - // metadata active timeline exists yet. + // the earliest commit on the dataset active timeline, + // the dataset active timeline has instants: + // 00000002.rollback, 00000007.commit, 00000008.commit assertEquals(9, metadataTableInstants.size()); assertTrue(metadataTableInstants.contains( new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000007001"))); @@ -1390,6 +1401,28 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPendingClusteringAfterArchiveCommit(boolean enableMetadata) throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 5, 2); + // timeline:0000000(completed)->00000001(completed)->00000002(replace&inflight)->00000003(completed)->...->00000007(completed) + HoodieTestDataGenerator.createPendingReplaceFile(basePath, "00000002", wrapperFs.getConf()); + for (int i = 1; i < 8; i++) { + if (i != 2) { + testTable.doWriteOperation("0000000" + i, WriteOperationType.CLUSTER, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2); + } + // archival + Pair, List> commitsList = archiveAndGetCommitsList(writeConfig); + List originalCommits = commitsList.getKey(); + List commitsAfterArchival = commitsList.getValue(); + assertEquals(originalCommits, commitsAfterArchival); + } + + HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); + assertEquals(6, timeline.countInstants(), + "Since we have a pending clustering instant at 00000002, we should never archive any commit after 00000000"); + } + private Pair, List> archiveAndGetCommitsList(HoodieWriteConfig writeConfig) throws IOException { metaClient.reloadActiveTimeline(); HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index cb7b2e6b3c43a..e1c2414e90eb4 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.HoodieSparkUtils; import org.apache.hudi.client.SparkRDDReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -106,6 +107,17 @@ public class SparkClientFunctionalTestHarness implements SparkProvider, HoodieMe @TempDir protected java.nio.file.Path tempDir; + public static Map getSparkSqlConf() { + Map sqlConf = new HashMap<>(); + sqlConf.put("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension"); + + if (HoodieSparkUtils.gteqSpark3_2()) { + sqlConf.put("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); + } + + return sqlConf; + } + public String basePath() { return tempDir.toAbsolutePath().toUri().toString(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java index 718fff3cf3a73..d784cdbfe02ed 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/HFileBootstrapIndex.java @@ -295,14 +295,16 @@ private HFile.Reader fileIdIndexReader() { @Override public List getIndexedPartitionPaths() { - HFileScanner scanner = partitionIndexReader().getScanner(true, false); - return getAllKeys(scanner, HFileBootstrapIndex::getPartitionFromKey); + try (HFileScanner scanner = partitionIndexReader().getScanner(true, false)) { + return getAllKeys(scanner, HFileBootstrapIndex::getPartitionFromKey); + } } @Override public List getIndexedFileGroupIds() { - HFileScanner scanner = fileIdIndexReader().getScanner(true, false); - return getAllKeys(scanner, HFileBootstrapIndex::getFileGroupFromKey); + try (HFileScanner scanner = fileIdIndexReader().getScanner(true, false)) { + return getAllKeys(scanner, HFileBootstrapIndex::getFileGroupFromKey); + } } private List getAllKeys(HFileScanner scanner, Function converter) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index 366d19fe6ebc0..86ff141ca7200 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -77,7 +77,13 @@ public void setDefaultValue(ConfigProperty configProperty) { if (configProperty.getInferFunc().isPresent()) { inferValue = configProperty.getInferFunc().get().apply(this); } - props.setProperty(configProperty.key(), inferValue.isPresent() ? inferValue.get().toString() : configProperty.defaultValue().toString()); + if (inferValue.isPresent() || configProperty.hasDefaultValue()) { + props.setProperty( + configProperty.key(), + inferValue.isPresent() + ? inferValue.get().toString() + : configProperty.defaultValue().toString()); + } } } @@ -121,7 +127,7 @@ protected void setDefaults(String configClassName) { .forEach(f -> { try { ConfigProperty cfgProp = (ConfigProperty) f.get("null"); - if (cfgProp.hasDefaultValue()) { + if (cfgProp.hasDefaultValue() || cfgProp.getInferFunc().isPresent()) { setDefaultValue(cfgProp); } } catch (IllegalAccessException e) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index f4895f988218c..18d4aaba09e15 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -79,7 +79,7 @@ public class FSUtils { private static final Logger LOG = LogManager.getLogger(FSUtils.class); // Log files are of this pattern - .b5068208-e1a4-11e6-bf01-fe55135034f3_20170101134598.log.1 public static final Pattern LOG_FILE_PATTERN = - Pattern.compile("\\.(.*)_(.*)\\.(.*)\\.([0-9]*)(_(([0-9]*)-([0-9]*)-([0-9]*)))?"); + Pattern.compile("^\\.(.+)_(.*)\\.(log|archive)\\.(\\d+)(_((\\d+)-(\\d+)-(\\d+)))?"); private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10; private static final long MIN_CLEAN_TO_KEEP = 10; private static final long MIN_ROLLBACK_TO_KEEP = 10; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java index 075f811a42ea7..f6450326fb1cb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java @@ -254,4 +254,14 @@ public boolean exists(Path f) throws IOException { public Configuration getConf() { return fileSystem.getConf(); } + + @Override + public short getDefaultReplication() { + return fileSystem.getDefaultReplication(); + } + + @Override + public short getDefaultReplication(Path path) { + return fileSystem.getDefaultReplication(path); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java index 9082d572a4bdb..e9c1e30a15263 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/debezium/AbstractDebeziumAvroPayload.java @@ -56,15 +56,19 @@ public AbstractDebeziumAvroPayload(Option record) { @Override public Option getInsertValue(Schema schema) throws IOException { - IndexedRecord insertRecord = getInsertRecord(schema); - return handleDeleteOperation(insertRecord); + Option insertValue = getInsertRecord(schema); + return insertValue.isPresent() ? handleDeleteOperation(insertValue.get()) : Option.empty(); } @Override public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { // Step 1: If the time occurrence of the current record in storage is higher than the time occurrence of the // insert record (including a delete record), pick the current record. - if (shouldPickCurrentRecord(currentValue, getInsertRecord(schema), schema)) { + Option insertValue = getInsertRecord(schema); + if (!insertValue.isPresent()) { + return Option.empty(); + } + if (shouldPickCurrentRecord(currentValue, insertValue.get(), schema)) { return Option.of(currentValue); } // Step 2: Pick the insert record (as a delete record if its a deleted event) @@ -84,7 +88,7 @@ private Option handleDeleteOperation(IndexedRecord insertRecord) return delete ? Option.empty() : Option.of(insertRecord); } - private IndexedRecord getInsertRecord(Schema schema) throws IOException { - return super.getInsertValue(schema).get(); + private Option getInsertRecord(Schema schema) throws IOException { + return super.getInsertValue(schema); } } \ No newline at end of file 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 16dd373486f61..64a38d1bb5469 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 @@ -58,6 +58,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -97,6 +98,10 @@ public class HoodieTableMetaClient implements Serializable { public static final String MARKER_EXTN = ".marker"; + // In-memory cache for archived timeline based on the start instant time + // Only one entry should be present in this map + private final Map archivedTimelineMap = new HashMap<>(); + // NOTE: Since those two parameters lay on the hot-path of a lot of computations, we // use tailored extension of the {@code Path} class allowing to avoid repetitive // computations secured by its immutability @@ -110,7 +115,6 @@ public class HoodieTableMetaClient implements Serializable { private TimelineLayoutVersion timelineLayoutVersion; protected HoodieTableConfig tableConfig; protected HoodieActiveTimeline activeTimeline; - private HoodieArchivedTimeline archivedTimeline; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); protected HoodieMetastoreConfig metastoreConfig; @@ -365,10 +369,7 @@ public FileSystemRetryConfig getFileSystemRetryConfig() { * @return Archived commit timeline */ public synchronized HoodieArchivedTimeline getArchivedTimeline() { - if (archivedTimeline == null) { - archivedTimeline = new HoodieArchivedTimeline(this); - } - return archivedTimeline; + return getArchivedTimeline(StringUtils.EMPTY_STRING); } public HoodieMetastoreConfig getMetastoreConfig() { @@ -379,21 +380,49 @@ public HoodieMetastoreConfig getMetastoreConfig() { } /** - * Returns fresh new archived commits as a timeline from startTs (inclusive). - * - *

This is costly operation if really early endTs is specified. - * Be caution to use this only when the time range is short. - * - *

This method is not thread safe. + * Returns the cached archived timeline from startTs (inclusive). * - * @return Archived commit timeline + * @param startTs The start instant time (inclusive) of the archived timeline. + * @return the archived timeline. */ public HoodieArchivedTimeline getArchivedTimeline(String startTs) { - return new HoodieArchivedTimeline(this, startTs); + return getArchivedTimeline(startTs, true); + } + + /** + * Returns the cached archived timeline if using in-memory cache or a fresh new archived + * timeline if not using cache, from startTs (inclusive). + *

+ * Instantiating an archived timeline is costly operation if really early startTs is + * specified. + *

+ * This method is not thread safe. + * + * @param startTs The start instant time (inclusive) of the archived timeline. + * @param useCache Whether to use in-memory cache. + * @return the archived timeline based on the arguments. + */ + public HoodieArchivedTimeline getArchivedTimeline(String startTs, boolean useCache) { + if (useCache) { + if (!archivedTimelineMap.containsKey(startTs)) { + // Only keep one entry in the map + archivedTimelineMap.clear(); + archivedTimelineMap.put(startTs, instantiateArchivedTimeline(startTs)); + } + return archivedTimelineMap.get(startTs); + } + return instantiateArchivedTimeline(startTs); + } + + private HoodieArchivedTimeline instantiateArchivedTimeline(String startTs) { + return StringUtils.isNullOrEmpty(startTs) + ? new HoodieArchivedTimeline(this) + : new HoodieArchivedTimeline(this, startTs); } /** * Validate table properties. + * * @param properties Properties from writeConfig. */ public void validateTableProperties(Properties properties) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java index a070a7e94d1c0..4642885202260 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineUtils.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; @@ -39,6 +40,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVEPOINT_ACTION; + /** * TimelineUtils provides a common way to query incremental meta-data changes for a hoodie table. * @@ -87,15 +93,15 @@ public static List getDroppedPartitions(HoodieTimeline timeline) { public static List getAffectedPartitions(HoodieTimeline timeline) { return timeline.filterCompletedInstants().getInstants().flatMap(s -> { switch (s.getAction()) { - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.DELTA_COMMIT_ACTION: + case COMMIT_ACTION: + case DELTA_COMMIT_ACTION: try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class); return commitMetadata.getPartitionToWriteStats().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions written at " + s, e); - } - case HoodieTimeline.REPLACE_COMMIT_ACTION: + } + case REPLACE_COMMIT_ACTION: try { HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( timeline.getInstantDetails(s).get(), HoodieReplaceCommitMetadata.class); @@ -148,11 +154,11 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { * Get extra metadata for specified key from latest commit/deltacommit/replacecommit(eg. insert_overwrite) instant. */ public static Option getExtraMetadataFromLatest(HoodieTableMetaClient metaClient, String extraMetadataKey) { - return metaClient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants() + return metaClient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants() // exclude clustering commits for returning user stored extra metadata .filter(instant -> !isClusteringCommit(metaClient, instant)) .findFirst().map(instant -> - getMetadataValue(metaClient, extraMetadataKey, instant)).orElse(Option.empty()); + getMetadataValue(metaClient, extraMetadataKey, instant)).orElse(Option.empty()); } /** @@ -170,7 +176,7 @@ public static Option getExtraMetadataFromLatestIncludeClustering(HoodieT */ public static Map> getAllExtraMetadataForKey(HoodieTableMetaClient metaClient, String extraMetadataKey) { return metaClient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants().collect(Collectors.toMap( - HoodieInstant::getTimestamp, instant -> getMetadataValue(metaClient, extraMetadataKey, instant))); + HoodieInstant::getTimestamp, instant -> getMetadataValue(metaClient, extraMetadataKey, instant))); } private static Option getMetadataValue(HoodieTableMetaClient metaClient, String extraMetadataKey, HoodieInstant instant) { @@ -184,10 +190,10 @@ private static Option getMetadataValue(HoodieTableMetaClient metaClient, throw new HoodieIOException("Unable to parse instant metadata " + instant, e); } } - + public static boolean isClusteringCommit(HoodieTableMetaClient metaClient, HoodieInstant instant) { try { - if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) { + if (REPLACE_COMMIT_ACTION.equals(instant.getAction())) { // replacecommit is used for multiple operations: insert_overwrite/cluster etc. // Check operation type to see if this instant is related to clustering. HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes( @@ -209,4 +215,84 @@ public static HoodieDefaultTimeline getTimeline(HoodieTableMetaClient metaClient } return activeTimeline; } + + /** + * Returns a Hudi timeline with commits after the given instant time (exclusive). + * + * @param metaClient {@link HoodieTableMetaClient} instance. + * @param exclusiveStartInstantTime Start instant time (exclusive). + * @return Hudi timeline. + */ + public static HoodieTimeline getCommitsTimelineAfter( + HoodieTableMetaClient metaClient, String exclusiveStartInstantTime) { + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + HoodieDefaultTimeline timeline = + activeTimeline.isBeforeTimelineStarts(exclusiveStartInstantTime) + ? metaClient.getArchivedTimeline(exclusiveStartInstantTime) + .mergeTimeline(activeTimeline) + : activeTimeline; + return timeline.getCommitsTimeline() + .findInstantsAfter(exclusiveStartInstantTime, Integer.MAX_VALUE); + } + + /** + * Returns the commit metadata of the given instant. + * + * @param instant The hoodie instant + * @param timeline The timeline + * @return the commit metadata + */ + public static HoodieCommitMetadata getCommitMetadata( + HoodieInstant instant, + HoodieTimeline timeline) throws IOException { + byte[] data = timeline.getInstantDetails(instant).get(); + if (instant.getAction().equals(REPLACE_COMMIT_ACTION)) { + return HoodieReplaceCommitMetadata.fromBytes(data, HoodieReplaceCommitMetadata.class); + } else { + return HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + } + } + + /** + * Gets the qualified earliest instant from the active timeline of the data table + * for the archival in metadata table. + *

+ * the qualified earliest instant is chosen as the earlier one between the earliest + * commit (COMMIT, DELTA_COMMIT, and REPLACE_COMMIT only, considering non-savepoint + * commit only if enabling archive beyond savepoint) and the earliest inflight + * instant (all actions). + * + * @param dataTableActiveTimeline the active timeline of the data table. + * @param shouldArchiveBeyondSavepoint whether to archive beyond savepoint. + * @return the instant meeting the requirement. + */ + public static Option getEarliestInstantForMetadataArchival( + HoodieActiveTimeline dataTableActiveTimeline, boolean shouldArchiveBeyondSavepoint) { + // This is for commits only, not including CLEAN, ROLLBACK, etc. + // When archive beyond savepoint is enabled, there are chances that there could be holes + // in the timeline due to archival and savepoint interplay. So, the first non-savepoint + // commit in the data timeline is considered as beginning of the active timeline. + Option earliestCommit = shouldArchiveBeyondSavepoint + ? dataTableActiveTimeline.getTimelineOfActions( + CollectionUtils.createSet( + COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION, SAVEPOINT_ACTION)) + .getFirstNonSavepointCommit() + : dataTableActiveTimeline.getCommitsTimeline().firstInstant(); + // This is for all instants which are in-flight + Option earliestInflight = + dataTableActiveTimeline.filterInflightsAndRequested().firstInstant(); + + if (earliestCommit.isPresent() && earliestInflight.isPresent()) { + if (earliestCommit.get().compareTo(earliestInflight.get()) < 0) { + return earliestCommit; + } + return earliestInflight; + } else if (earliestCommit.isPresent()) { + return earliestCommit; + } else if (earliestInflight.isPresent()) { + return earliestInflight; + } else { + return Option.empty(); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 39bb3a2a5da27..69482fd5db6d5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -38,6 +38,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hadoop.fs.FileStatus; @@ -52,6 +53,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -60,6 +62,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; +import java.util.function.Function; import java.util.function.Predicate; import java.util.regex.Matcher; import java.util.stream.Collectors; @@ -91,8 +94,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV // Locks to control concurrency. Sync operations use write-lock blocking all fetch operations. // For the common-case, we allow concurrent read of single or multiple partitions private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock(); - private final ReadLock readLock = globalLock.readLock(); - private final WriteLock writeLock = globalLock.writeLock(); + protected final ReadLock readLock = globalLock.readLock(); + protected final WriteLock writeLock = globalLock.writeLock(); private BootstrapIndex bootstrapIndex; @@ -117,7 +120,7 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi /** * Refresh commits timeline. - * + * * @param visibleActiveTimeline Visible Active Timeline */ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { @@ -159,13 +162,13 @@ public List addFilesToView(FileStatus[] statuses) { * Build FileGroups from passed in file-status. */ protected List buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline, - boolean addPendingCompactionFileSlice) { + boolean addPendingCompactionFileSlice) { return buildFileGroups(convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline, addPendingCompactionFileSlice); } protected List buildFileGroups(Stream baseFileStream, - Stream logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { + Stream logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { Map, List> baseFiles = baseFileStream.collect(Collectors.groupingBy(baseFile -> { String partitionPathStr = getPartitionPathFor(baseFile); @@ -223,7 +226,7 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { // get replace instant mapping for each partition, fileId return replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().flatMap(entry -> entry.getValue().stream().map(e -> - new AbstractMap.SimpleEntry<>(new HoodieFileGroupId(entry.getKey(), e), instant))); + new AbstractMap.SimpleEntry<>(new HoodieFileGroupId(entry.getKey(), e), instant))); } catch (HoodieIOException ex) { if (ex.getIOException() instanceof FileNotFoundException) { @@ -258,6 +261,8 @@ public void close() { /** * Clears the partition Map and reset view states. + *

+ * NOTE: The logic MUST BE guarded by the write lock. */ @Override public void reset() { @@ -274,7 +279,7 @@ public void reset() { /** * Clear the resource. */ - private void clear() { + protected void clear() { addedPartitions.clear(); resetViewState(); bootstrapIndex = null; @@ -285,6 +290,114 @@ private void clear() { */ protected abstract void resetViewState(); + /** + * Batch loading all the partitions if needed. + * + * @return A list of relative partition paths of all partitions. + */ + private List ensureAllPartitionsLoadedCorrectly() { + ValidationUtils.checkArgument(!isClosed(), "View is already closed"); + try { + List formattedPartitionList = getAllPartitionPaths().stream() + .map(this::formatPartitionKey).collect(Collectors.toList()); + ensurePartitionsLoadedCorrectly(formattedPartitionList); + return formattedPartitionList; + } catch (IOException e) { + throw new HoodieIOException("Failed to get all partition paths", e); + } + } + + /** + * Allows lazily loading the partitions if needed. + * + * @param partitionList list of partitions to be loaded if not present. + */ + private void ensurePartitionsLoadedCorrectly(List partitionList) { + + ValidationUtils.checkArgument(!isClosed(), "View is already closed"); + + Set partitionSet = new HashSet<>(); + synchronized (addedPartitions) { + partitionList.forEach(partition -> { + if (!addedPartitions.containsKey(partition) && !isPartitionAvailableInStore(partition)) { + partitionSet.add(partition); + } + }); + + if (!partitionSet.isEmpty()) { + long beginTs = System.currentTimeMillis(); + // Not loaded yet + try { + LOG.info("Building file system view for partitions " + partitionSet); + + // Pairs of relative partition path and absolute partition path + List> absolutePartitionPathList = partitionSet.stream() + .map(partition -> Pair.of( + partition, FSUtils.getPartitionPath(metaClient.getBasePathV2(), partition))) + .collect(Collectors.toList()); + long beginLsTs = System.currentTimeMillis(); + Map, FileStatus[]> statusesMap = + listPartitions(absolutePartitionPathList); + long endLsTs = System.currentTimeMillis(); + LOG.debug("Time taken to list partitions " + partitionSet + " =" + (endLsTs - beginLsTs)); + statusesMap.forEach((partitionPair, statuses) -> { + String relativePartitionStr = partitionPair.getLeft(); + List groups = addFilesToView(statuses); + if (groups.isEmpty()) { + storePartitionView(relativePartitionStr, new ArrayList<>()); + } + LOG.debug("#files found in partition (" + relativePartitionStr + ") =" + statuses.length); + }); + } catch (IOException e) { + throw new HoodieIOException("Failed to list base files in partitions " + partitionSet, e); + } + long endTs = System.currentTimeMillis(); + LOG.debug("Time to load partition " + partitionSet + " =" + (endTs - beginTs)); + } + + partitionSet.forEach(partition -> + addedPartitions.computeIfAbsent(partition, partitionPathStr -> true) + ); + } + } + + /*** + * @return A list of relative partition paths of all partitions. + * @throws IOException upon error. + */ + protected List getAllPartitionPaths() throws IOException { + throw new HoodieException("Getting all partition paths with file system listing sequentially " + + "can be very slow. This should not be invoked."); + } + + /** + * @param partitionPathList A list of pairs of the relative and absolute paths of the partitions. + * @return all the files from the partitions. + * @throws IOException upon error. + */ + protected Map, FileStatus[]> listPartitions( + List> partitionPathList) throws IOException { + Map, FileStatus[]> fileStatusMap = new HashMap<>(); + + for (Pair partitionPair : partitionPathList) { + Path absolutePartitionPath = partitionPair.getRight(); + try { + fileStatusMap.put(partitionPair, metaClient.getFs().listStatus(absolutePartitionPath)); + } catch (IOException e) { + // Create the path if it does not exist already + if (!metaClient.getFs().exists(absolutePartitionPath)) { + metaClient.getFs().mkdirs(absolutePartitionPath); + fileStatusMap.put(partitionPair, new FileStatus[0]); + } else { + // in case the partition path was created by another caller + fileStatusMap.put(partitionPair, metaClient.getFs().listStatus(absolutePartitionPath)); + } + } + } + + return fileStatusMap; + } + /** * Allows lazily loading the partitions if needed. * @@ -523,19 +636,38 @@ public final Stream getLatestBaseFilesBeforeOrOn(String partitio readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchAllStoredFileGroups(partitionPath) - .filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime)) - .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles() - .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime - )) - .filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst())) - .filter(Option::isPresent).map(Option::get) - .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); + return getLatestBaseFilesBeforeOrOnFromCache(partitionPath, maxCommitTime); + } finally { + readLock.unlock(); + } + } + + @Override + public final Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) { + try { + readLock.lock(); + + List formattedPartitionList = ensureAllPartitionsLoadedCorrectly(); + return formattedPartitionList.stream().collect(Collectors.toMap( + Function.identity(), + partitionPath -> getLatestBaseFilesBeforeOrOnFromCache(partitionPath, maxCommitTime) + )); } finally { readLock.unlock(); } } + private Stream getLatestBaseFilesBeforeOrOnFromCache(String partitionPath, String maxCommitTime) { + return fetchAllStoredFileGroups(partitionPath) + .filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime)) + .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles() + .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime + )) + .filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst())) + .filter(Option::isPresent).map(Option::get) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); + } + @Override public final Option getBaseFileOn(String partitionStr, String instantTime, String fileId) { try { @@ -546,8 +678,8 @@ public final Option getBaseFileOn(String partitionStr, String in return Option.empty(); } else { return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() - .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, - instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null)) + .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, + instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null)) .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } } finally { @@ -582,8 +714,8 @@ public final Stream getLatestBaseFilesInRange(List commi return fetchAllStoredFileGroups() .filter(fileGroup -> !isFileGroupReplacedBeforeAny(fileGroup.getFileGroupId(), commitsToReturn)) .map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional( - fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime()) - && !isBaseFileDueToPendingCompaction(baseFile) && !isBaseFileDueToPendingClustering(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent()) + fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime()) + && !isBaseFileDueToPendingCompaction(baseFile) && !isBaseFileDueToPendingClustering(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent()) .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); } finally { readLock.unlock(); @@ -613,9 +745,9 @@ public final Stream getLatestFileSlices(String partitionStr) { String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchLatestFileSlices(partitionPath) - .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) - .flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, true)) - .map(this::addBootstrapBaseFileIfPresent); + .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) + .flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, true)) + .map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -670,26 +802,26 @@ public final Stream getLatestUnCompactedFileSlices(String partitionSt @Override public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr, String maxCommitTime, - boolean includeFileSlicesInPendingCompaction) { + boolean includeFileSlicesInPendingCompaction) { try { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); Stream> allFileSliceStream = fetchAllStoredFileGroups(partitionPath) - .filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime)) - .map(fg -> fg.getAllFileSlicesBeforeOn(maxCommitTime)); + .filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime)) + .map(fg -> fg.getAllFileSlicesBeforeOn(maxCommitTime)); if (includeFileSlicesInPendingCompaction) { return allFileSliceStream.map(sliceStream -> sliceStream.flatMap(slice -> this.filterBaseFileAfterPendingCompaction(slice, false))) - .map(sliceStream -> Option.fromJavaOptional(sliceStream.findFirst())).filter(Option::isPresent).map(Option::get) - .map(this::addBootstrapBaseFileIfPresent); + .map(sliceStream -> Option.fromJavaOptional(sliceStream.findFirst())).filter(Option::isPresent).map(Option::get) + .map(this::addBootstrapBaseFileIfPresent); } else { return allFileSliceStream - .map(sliceStream -> - Option.fromJavaOptional(sliceStream - .filter(slice -> !isPendingCompactionScheduledForFileId(slice.getFileGroupId())) - .filter(slice -> !slice.isEmpty()) - .findFirst())) - .filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); + .map(sliceStream -> + Option.fromJavaOptional(sliceStream + .filter(slice -> !isPendingCompactionScheduledForFileId(slice.getFileGroupId())) + .filter(slice -> !slice.isEmpty()) + .findFirst())) + .filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); } } finally { readLock.unlock(); @@ -1148,32 +1280,25 @@ public HoodieTimeline getTimeline() { return visibleCommitsAndCompactionTimeline; } + /** + * Syncs the file system view from storage to memory. Performs complete reset of file-system + * view. Subsequent partition view calls will load file slices against the latest timeline. + *

+ * NOTE: The logic MUST BE guarded by the write lock. + */ @Override public void sync() { - HoodieTimeline oldTimeline = getTimeline(); - HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); try { writeLock.lock(); - runSync(oldTimeline, newTimeline); + HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); + clear(); + // Initialize with new Hoodie timeline. + init(metaClient, newTimeline); } finally { writeLock.unlock(); } } - /** - * Performs complete reset of file-system view. Subsequent partition view calls will load file slices against latest - * timeline - * - * @param oldTimeline Old Hoodie Timeline - * @param newTimeline New Hoodie Timeline - */ - protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { - refreshTimeline(newTimeline); - clear(); - // Initialize with new Hoodie timeline. - init(metaClient, newTimeline); - } - /** * Return Only Commits and Compaction timeline for building file-groups. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index 9f9fd1f7ce601..04198462a4334 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -77,7 +77,18 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { } @Override - protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { + public void sync() { + try { + writeLock.lock(); + maySyncIncrementally(); + } finally { + writeLock.unlock(); + } + } + + protected void maySyncIncrementally() { + HoodieTimeline oldTimeline = getTimeline(); + HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); try { if (incrementalTimelineSyncEnabled) { TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline); @@ -93,14 +104,15 @@ protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { } catch (Exception ioe) { LOG.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe); } - - super.runSync(oldTimeline, newTimeline); + clear(); + // Initialize with new Hoodie timeline. + init(metaClient, newTimeline); } /** * Run incremental sync based on the diff result produced. * - * @param timeline New Timeline + * @param timeline New Timeline * @param diffResult Timeline Diff Result */ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index ff44c7cef017b..c7d885ca3476a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -39,6 +39,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; import java.util.stream.Stream; /** @@ -145,6 +146,12 @@ public Stream getLatestBaseFilesBeforeOrOn(String partitionPath, secondaryView::getLatestBaseFilesBeforeOrOn); } + @Override + public Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) { + return execute(maxCommitTime, preferredView::getAllLatestBaseFilesBeforeOrOn, + secondaryView::getAllLatestBaseFilesBeforeOrOn); + } + @Override public Option getLatestBaseFile(String partitionPath, String fileId) { return execute(partitionPath, fileId, preferredView::getLatestBaseFile, secondaryView::getLatestBaseFile); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index bd18ba22a25d6..c685664580418 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -54,6 +54,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -87,6 +88,8 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, String.format("%s/%s", BASE_URL, "datafiles/range/latest/"); public static final String LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL = String.format("%s/%s", BASE_URL, "datafiles/beforeoron/latest/"); + public static final String ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL = + String.format("%s/%s", BASE_URL, "basefiles/all/beforeoron/"); public static final String ALL_FILEGROUPS_FOR_PARTITION_URL = String.format("%s/%s", BASE_URL, "filegroups/all/partition/"); @@ -154,17 +157,17 @@ public RemoteHoodieTableFileSystemView(HoodieTableMetaClient metaClient, FileSys this.serverPort = viewConf.getRemoteViewServerPort(); this.timeoutMs = viewConf.getRemoteTimelineClientTimeoutSecs() * 1000; if (viewConf.isRemoteTimelineClientRetryEnabled()) { - retryHelper = new RetryHelper( - viewConf.getRemoteTimelineClientMaxRetryIntervalMs(), - viewConf.getRemoteTimelineClientMaxRetryNumbers(), - viewConf.getRemoteTimelineInitialRetryIntervalMs(), - viewConf.getRemoteTimelineClientRetryExceptions(), - "Sending request"); + retryHelper = new RetryHelper( + viewConf.getRemoteTimelineClientMaxRetryIntervalMs(), + viewConf.getRemoteTimelineClientMaxRetryNumbers(), + viewConf.getRemoteTimelineInitialRetryIntervalMs(), + viewConf.getRemoteTimelineClientRetryExceptions(), + "Sending request"); } } private T executeRequest(String requestPath, Map queryParameters, TypeReference reference, - RequestMethod method) throws IOException { + RequestMethod method) throws IOException { ValidationUtils.checkArgument(!closed, "View already closed"); URIBuilder builder = @@ -251,13 +254,36 @@ public Stream getLatestBaseFilesBeforeOrOn(String partitionPath, return getLatestBaseFilesFromParams(paramsMap, LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL); } + @Override + public Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) { + Map paramsMap = new HashMap<>(); + paramsMap.put(BASEPATH_PARAM, basePath); + paramsMap.put(MAX_INSTANT_PARAM, maxCommitTime); + + try { + Map> dataFileMap = executeRequest( + ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL, + paramsMap, + new TypeReference>>() { + }, + RequestMethod.GET); + return dataFileMap.entrySet().stream().collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().stream().map(BaseFileDTO::toHoodieBaseFile))); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + @Override public Option getBaseFileOn(String partitionPath, String instantTime, String fileId) { Map paramsMap = getParamsWithAdditionalParams(partitionPath, new String[] {INSTANT_PARAM, FILEID_PARAM}, new String[] {instantTime, fileId}); try { List dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + new TypeReference>() { + }, RequestMethod.GET); return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); @@ -492,7 +518,8 @@ public Option getLatestBaseFile(String partitionPath, String fil Map paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId); try { List dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap, - new TypeReference>() {}, RequestMethod.GET); + new TypeReference>() { + }, RequestMethod.GET); return Option.fromJavaOptional(dataFiles.stream().map(BaseFileDTO::toHoodieBaseFile).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index c32e2cabb1012..3d27e85d54f97 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.util.collection.Pair; import java.util.List; +import java.util.Map; import java.util.stream.Stream; /** @@ -64,6 +65,16 @@ interface BaseFileOnlyViewWithLatestSlice { */ Stream getLatestBaseFilesBeforeOrOn(String partitionPath, String maxCommitTime); + /** + * Streams the latest version base files in all partitions with precondition that + * commitTime(file) before maxCommitTime. + * + * @param maxCommitTime The max commit time to consider. + * @return A {@link Map} of partition path to the latest version base files before or on the + * commit time + */ + Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime); + /** * Stream all the latest data files pass. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java index 9d741a03f82ec..efbae1cd461ff 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ClusteringUtils.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -223,4 +224,53 @@ public static List getPendingClusteringInstantTimes(HoodieTableMe public static boolean isPendingClusteringInstant(HoodieTableMetaClient metaClient, HoodieInstant instant) { return getClusteringPlan(metaClient, instant).isPresent(); } + + /** + * Returns the oldest instant to retain between the clustering instant that there is such cleaning action or empty, + * and the latest instant before the oldest inflight clustering instant. + * + *

Checks whether the latest clustering instant has a subsequent cleaning action, and the oldest inflight clustering instant has a previous commit. + * + * @param activeTimeline The active timeline + * @param metaClient The meta client + * @return the oldest instant to retain for clustering + */ + public static Option getOldestInstantToRetainForClustering( + HoodieActiveTimeline activeTimeline, HoodieTableMetaClient metaClient) throws IOException { + Option oldestInstantToRetain = Option.empty(); + HoodieTimeline replaceTimeline = activeTimeline.getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.REPLACE_COMMIT_ACTION)); + if (!replaceTimeline.empty()) { + Option cleanInstantOpt = + activeTimeline.getCleanerTimeline().filter(instant -> !instant.isCompleted()).firstInstant(); + if (cleanInstantOpt.isPresent()) { + // The first clustering instant of which timestamp is greater than or equal to the earliest commit to retain of + // the clean metadata. + HoodieInstant cleanInstant = cleanInstantOpt.get(); + String earliestCommitToRetain = + CleanerUtils.getCleanerPlan(metaClient, + cleanInstant.isRequested() + ? cleanInstant + : HoodieTimeline.getCleanRequestedInstant(cleanInstant.getTimestamp())) + .getEarliestInstantToRetain().getTimestamp(); + if (!StringUtils.isNullOrEmpty(earliestCommitToRetain)) { + oldestInstantToRetain = replaceTimeline.filterCompletedInstants() + .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestCommitToRetain)) + .firstInstant(); + } + } + Option pendingInstantOpt = replaceTimeline.filterInflights().firstInstant(); + if (pendingInstantOpt.isPresent()) { + // Get the previous commit before the first inflight clustering instant. + Option beforePendingInstant = activeTimeline.getCommitsTimeline() + .filterCompletedInstants() + .findInstantsBefore(pendingInstantOpt.get().getTimestamp()) + .lastInstant(); + if (beforePendingInstant.isPresent() + && oldestInstantToRetain.map(instant -> instant.compareTo(beforePendingInstant.get()) > 0).orElse(true)) { + oldestInstantToRetain = beforePendingInstant; + } + } + } + return oldestInstantToRetain; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index 90d6e6ae90fb0..49899adf490ea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -62,6 +62,14 @@ public static boolean nonEmpty(Collection c) { return !isNullOrEmpty(c); } + public static U reduce(Collection c, U identity, BiFunction reducer) { + return c.stream() + .sequential() + .reduce(identity, reducer, (a, b) -> { + throw new UnsupportedOperationException(); + }); + } + /** * Makes a copy of provided {@link Properties} object */ @@ -111,7 +119,6 @@ public static T[] append(T[] array, T elem) { return combined; } - /** * Combines provided {@link List}s into one, returning new instance of {@link ArrayList} */ 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 fe40d98594287..417815e960c35 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 @@ -44,6 +44,7 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; +import java.net.URI; import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedList; @@ -69,7 +70,7 @@ public class RocksDBDAO { public RocksDBDAO(String basePath, String rocksDBBasePath) { this.rocksDBBasePath = - String.format("%s/%s/%s", rocksDBBasePath, basePath.replace("/", "_"), UUID.randomUUID().toString()); + String.format("%s/%s/%s", rocksDBBasePath, URI.create(basePath).getPath().replace(":","").replace("/", "_"), UUID.randomUUID().toString()); init(); totalBytesWritten = 0L; } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java index 659612cd5cfaf..6e1b000f2a11b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/InternalSchema.java @@ -25,6 +25,7 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -62,6 +63,10 @@ public InternalSchema(List columns) { this(DEFAULT_VERSION_ID, columns); } + public InternalSchema(RecordType recordType) { + this(DEFAULT_VERSION_ID, recordType); + } + public InternalSchema(Field... columns) { this(DEFAULT_VERSION_ID, Arrays.asList(columns)); } @@ -74,6 +79,18 @@ public InternalSchema(long versionId, List cols) { maxColumnId = idToName.isEmpty() ? -1 : idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get(); } + public InternalSchema(long versionId, RecordType recordType) { + this.versionId = versionId; + this.record = recordType; + this.idToName = recordType.fields().isEmpty() + ? Collections.emptyMap() + : InternalSchemaBuilder.getBuilder().buildIdToName(record); + this.nameToId = recordType.fields().isEmpty() + ? Collections.emptyMap() + : idToName.entrySet().stream().collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + this.maxColumnId = idToName.isEmpty() ? -1 : idToName.keySet().stream().max(Comparator.comparing(Integer::valueOf)).get(); + } + public InternalSchema(long versionId, int maxColumnId, List cols) { this.maxColumnId = maxColumnId; this.versionId = versionId; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java index fff10a700f618..7985f55dd8062 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/Types.java @@ -459,23 +459,30 @@ public int hashCode() { public static class RecordType extends NestedType { public static RecordType get(List fields) { - return new RecordType(fields); + return new RecordType(fields, null); + } + + public static RecordType get(List fields, String recordName) { + return new RecordType(fields, recordName); } public static RecordType get(Field... fields) { - return new RecordType(Arrays.asList(fields)); + return new RecordType(Arrays.asList(fields), null); } + // NOTE: This field is necessary to provide for lossless conversion b/w Avro and + // InternalSchema and back (Avro unfortunately relies not only on structural equivalence of + // schemas but also corresponding Record type's "name" when evaluating their compatibility); + // This field is nullable + private final String name; private final Field[] fields; private transient Map nameToFields = null; private transient Map idToFields = null; - private RecordType(List fields) { - this.fields = new Field[fields.size()]; - for (int i = 0; i < this.fields.length; i += 1) { - this.fields[i] = fields.get(i); - } + private RecordType(List fields, String name) { + this.name = name; + this.fields = fields.toArray(new Field[0]); } @Override @@ -513,6 +520,10 @@ public Type fieldType(String name) { return null; } + public String name() { + return name; + } + @Override public TypeID typeId() { return TypeID.RECORD; diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java index d941b27328aba..2427f466004d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/AvroInternalSchemaConverter.java @@ -22,6 +22,8 @@ import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; + +import org.apache.hudi.common.util.Option; import org.apache.hudi.internal.schema.HoodieSchemaException; import org.apache.hudi.internal.schema.InternalSchema; import org.apache.hudi.internal.schema.Type; @@ -43,6 +45,18 @@ */ public class AvroInternalSchemaConverter { + // NOTE: We're using dot as field's name delimiter for nested fields + // so that Avro is able to interpret qualified name as rather + // the combination of the Avro's namespace and actual record's name. + // For example qualified nested field's name "trip.fare.amount", + // Avro will produce a record with + // - Namespace: "trip.fare" + // - Name: "amount" + // + // This is crucial aspect of maintaining compatibility b/w schemas, after + // converting Avro [[Schema]]s to [[InternalSchema]]s and back + private static final String AVRO_NAME_DELIMITER = "."; + /** * Convert internalSchema to avro Schema. * @@ -50,12 +64,8 @@ public class AvroInternalSchemaConverter { * @param tableName the record name. * @return an avro Schema. */ - public static Schema convert(InternalSchema internalSchema, String tableName, String namespace) { - return buildAvroSchemaFromInternalSchema(internalSchema, tableName, namespace); - } - public static Schema convert(InternalSchema internalSchema, String tableName) { - return buildAvroSchemaFromInternalSchema(internalSchema, tableName, ""); + return buildAvroSchemaFromInternalSchema(internalSchema, tableName); } /** @@ -245,7 +255,7 @@ private static Type visitAvroPrimitiveToBuildInternalType(Schema primitive) { */ public static Schema buildAvroSchemaFromType(Type type, String recordName) { Map cache = new HashMap<>(); - return visitInternalSchemaToBuildAvroSchema(type, cache, recordName, ""); + return visitInternalSchemaToBuildAvroSchema(type, cache, recordName); } /** @@ -255,9 +265,9 @@ public static Schema buildAvroSchemaFromType(Type type, String recordName) { * @param recordName the record name * @return a Avro schema match hudi internal schema. */ - public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, String recordName, String namespace) { + public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, String recordName) { Map cache = new HashMap<>(); - return visitInternalSchemaToBuildAvroSchema(schema.getRecord(), cache, recordName, namespace); + return visitInternalSchemaToBuildAvroSchema(schema.getRecord(), cache, recordName); } /** @@ -268,15 +278,14 @@ public static Schema buildAvroSchemaFromInternalSchema(InternalSchema schema, St * @param recordName the record name * @return a Avro schema match this type */ - private static Schema visitInternalSchemaToBuildAvroSchema( - Type type, Map cache, String recordName, String namespace) { + private static Schema visitInternalSchemaToBuildAvroSchema(Type type, Map cache, String recordName) { switch (type.typeId()) { case RECORD: Types.RecordType record = (Types.RecordType) type; List schemas = new ArrayList<>(); record.fields().forEach(f -> { - Schema tempSchema = visitInternalSchemaToBuildAvroSchema( - f.type(), cache, recordName + "_" + f.name(), namespace); + String nestedRecordName = recordName + AVRO_NAME_DELIMITER + f.name(); + Schema tempSchema = visitInternalSchemaToBuildAvroSchema(f.type(), cache, nestedRecordName); // convert tempSchema Schema result = f.isOptional() ? AvroInternalSchemaConverter.nullableSchema(tempSchema) : tempSchema; schemas.add(result); @@ -287,13 +296,13 @@ private static Schema visitInternalSchemaToBuildAvroSchema( if (recordSchema != null) { return recordSchema; } - recordSchema = visitInternalRecordToBuildAvroRecord(record, schemas, recordName, namespace); + recordSchema = visitInternalRecordToBuildAvroRecord(record, schemas, recordName); cache.put(record, recordSchema); return recordSchema; case ARRAY: Types.ArrayType array = (Types.ArrayType) type; Schema elementSchema; - elementSchema = visitInternalSchemaToBuildAvroSchema(array.elementType(), cache, recordName, namespace); + elementSchema = visitInternalSchemaToBuildAvroSchema(array.elementType(), cache, recordName); Schema arraySchema; arraySchema = cache.get(array); if (arraySchema != null) { @@ -306,8 +315,8 @@ private static Schema visitInternalSchemaToBuildAvroSchema( Types.MapType map = (Types.MapType) type; Schema keySchema; Schema valueSchema; - keySchema = visitInternalSchemaToBuildAvroSchema(map.keyType(), cache, recordName, namespace); - valueSchema = visitInternalSchemaToBuildAvroSchema(map.valueType(), cache, recordName, namespace); + keySchema = visitInternalSchemaToBuildAvroSchema(map.keyType(), cache, recordName); + valueSchema = visitInternalSchemaToBuildAvroSchema(map.valueType(), cache, recordName); Schema mapSchema; mapSchema = cache.get(map); if (mapSchema != null) { @@ -317,7 +326,7 @@ private static Schema visitInternalSchemaToBuildAvroSchema( cache.put(map, mapSchema); return mapSchema; default: - Schema primitiveSchema = visitInternalPrimitiveToBuildAvroPrimitiveType((Type.PrimitiveType) type); + Schema primitiveSchema = visitInternalPrimitiveToBuildAvroPrimitiveType((Type.PrimitiveType) type, recordName); cache.put(type, primitiveSchema); return primitiveSchema; } @@ -327,16 +336,16 @@ private static Schema visitInternalSchemaToBuildAvroSchema( * Converts hudi RecordType to Avro RecordType. * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema */ - private static Schema visitInternalRecordToBuildAvroRecord( - Types.RecordType record, List fieldSchemas, String recordName, String namespace) { - List fields = record.fields(); + private static Schema visitInternalRecordToBuildAvroRecord(Types.RecordType recordType, List fieldSchemas, String recordNameFallback) { + List fields = recordType.fields(); List avroFields = new ArrayList<>(); for (int i = 0; i < fields.size(); i++) { Types.Field f = fields.get(i); Schema.Field field = new Schema.Field(f.name(), fieldSchemas.get(i), f.doc(), f.isOptional() ? JsonProperties.NULL_VALUE : null); avroFields.add(field); } - return Schema.createRecord(recordName, null, namespace, false, avroFields); + String recordName = Option.ofNullable(recordType.name()).orElse(recordNameFallback); + return Schema.createRecord(recordName, null, null, false, avroFields); } /** @@ -371,62 +380,68 @@ private static Schema visitInternalMapToBuildAvroMap(Types.MapType map, Schema k * Converts hudi PrimitiveType to Avro PrimitiveType. * this is auxiliary function used by visitInternalSchemaToBuildAvroSchema */ - private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.PrimitiveType primitive) { - Schema primitiveSchema; + private static Schema visitInternalPrimitiveToBuildAvroPrimitiveType(Type.PrimitiveType primitive, String recordName) { switch (primitive.typeId()) { case BOOLEAN: - primitiveSchema = Schema.create(Schema.Type.BOOLEAN); - break; + return Schema.create(Schema.Type.BOOLEAN); + case INT: - primitiveSchema = Schema.create(Schema.Type.INT); - break; + return Schema.create(Schema.Type.INT); + case LONG: - primitiveSchema = Schema.create(Schema.Type.LONG); - break; + return Schema.create(Schema.Type.LONG); + case FLOAT: - primitiveSchema = Schema.create(Schema.Type.FLOAT); - break; + return Schema.create(Schema.Type.FLOAT); + case DOUBLE: - primitiveSchema = Schema.create(Schema.Type.DOUBLE); - break; + return Schema.create(Schema.Type.DOUBLE); + case DATE: - primitiveSchema = LogicalTypes.date() - .addToSchema(Schema.create(Schema.Type.INT)); - break; + return LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + case TIME: - primitiveSchema = LogicalTypes.timeMicros() - .addToSchema(Schema.create(Schema.Type.LONG)); - break; + return LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case TIMESTAMP: - primitiveSchema = LogicalTypes.timestampMicros() - .addToSchema(Schema.create(Schema.Type.LONG)); - break; + return LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + case STRING: - primitiveSchema = Schema.create(Schema.Type.STRING); - break; - case UUID: - primitiveSchema = LogicalTypes.uuid() - .addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)); - break; - case FIXED: - Types.FixedType fixed = (Types.FixedType) primitive; - primitiveSchema = Schema.createFixed("fixed_" + fixed.getFixedSize(), null, null, fixed.getFixedSize()); - break; + return Schema.create(Schema.Type.STRING); + case BINARY: - primitiveSchema = Schema.create(Schema.Type.BYTES); - break; - case DECIMAL: + return Schema.create(Schema.Type.BYTES); + + case UUID: { + // NOTE: All schemas corresponding to Avro's type [[FIXED]] are generated + // with the "fixed" name to stay compatible w/ [[SchemaConverters]] + String name = recordName + AVRO_NAME_DELIMITER + "fixed"; + Schema fixedSchema = Schema.createFixed(name, null, null, 16); + return LogicalTypes.uuid().addToSchema(fixedSchema); + } + + case FIXED: { + Types.FixedType fixed = (Types.FixedType) primitive; + // NOTE: All schemas corresponding to Avro's type [[FIXED]] are generated + // with the "fixed" name to stay compatible w/ [[SchemaConverters]] + String name = recordName + AVRO_NAME_DELIMITER + "fixed"; + return Schema.createFixed(name, null, null, fixed.getFixedSize()); + } + + case DECIMAL: { Types.DecimalType decimal = (Types.DecimalType) primitive; - primitiveSchema = LogicalTypes.decimal(decimal.precision(), decimal.scale()) - .addToSchema(Schema.createFixed( - "decimal_" + decimal.precision() + "_" + decimal.scale(), - null, null, computeMinBytesForPrecision(decimal.precision()))); - break; + // NOTE: All schemas corresponding to Avro's type [[FIXED]] are generated + // with the "fixed" name to stay compatible w/ [[SchemaConverters]] + String name = recordName + AVRO_NAME_DELIMITER + "fixed"; + Schema fixedSchema = Schema.createFixed(name, null, null, computeMinBytesForPrecision(decimal.precision())); + return LogicalTypes.decimal(decimal.precision(), decimal.scale()) + .addToSchema(fixedSchema); + } + default: throw new UnsupportedOperationException( "Unsupported type ID: " + primitive.typeId()); } - return primitiveSchema; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java index 413a3c4df1bc3..c143aaa6f92b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/AvroSchemaEvolutionUtils.java @@ -29,6 +29,8 @@ import java.util.TreeMap; import java.util.stream.Collectors; +import static org.apache.hudi.common.util.CollectionUtils.reduce; + /** * Utility methods to support evolve old avro schema based on a given schema. */ @@ -126,13 +128,11 @@ public static Schema canonicalizeColumnNullability(Schema writeSchema, Schema re if (candidateUpdateCols.isEmpty()) { return writeSchema; } - // try to correct all changes - TableChanges.ColumnUpdateChange updateChange = TableChanges.ColumnUpdateChange.get(writeInternalSchema); - candidateUpdateCols.stream().forEach(f -> updateChange.updateColumnNullability(f, true)); - Schema result = AvroInternalSchemaConverter.convert( - SchemaChangeUtils.applyTableChanges2Schema(writeInternalSchema, updateChange), - writeSchema.getName(), writeSchema.getNamespace()); - return result; + // Reconcile nullability constraints (by executing phony schema change) + TableChanges.ColumnUpdateChange schemaChange = reduce(candidateUpdateCols, TableChanges.ColumnUpdateChange.get(writeInternalSchema), + (change, field) -> change.updateColumnNullability(field, true)); + + return AvroInternalSchemaConverter.convert(SchemaChangeUtils.applyTableChanges2Schema(writeInternalSchema, schemaChange), writeSchema.getFullName()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java index d719008042021..d18bd87dfde42 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/SchemaChangeUtils.java @@ -239,7 +239,7 @@ public static InternalSchema applyTableChanges2Schema(InternalSchema internalSch // deal with root level changes List newFields = TableChangesHelper.applyAddChange2Fields(newType.fields(), new ArrayList<>(), updates.getPositionChangeMap().get(-1)); - return new InternalSchema(newFields); + return new InternalSchema(Types.RecordType.get(newFields, newType.name())); } /** 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 187791558a24a..698982a02cfa1 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 @@ -22,6 +22,7 @@ import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; @@ -543,10 +544,21 @@ private boolean isFullScanAllowedForPartition(String partitionName) { */ private List getRollbackedCommits(HoodieInstant instant, HoodieActiveTimeline timeline) { try { + List commitsToRollback = null; if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { - HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( - timeline.getInstantDetails(instant).get()); - return rollbackMetadata.getCommitsRollback(); + try { + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + timeline.getInstantDetails(instant).get()); + commitsToRollback = rollbackMetadata.getCommitsRollback(); + } catch (IOException e) { + // if file is empty, fetch the commits to rollback from rollback.requested file + HoodieRollbackPlan rollbackPlan = TimelineMetadataUtils.deserializeAvroMetadata( + timeline.readRollbackInfoAsBytes(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, + instant.getTimestamp())).get(), HoodieRollbackPlan.class); + commitsToRollback = Collections.singletonList(rollbackPlan.getInstantToRollback().getCommitTime()); + LOG.warn("Had to fetch rollback info from requested instant since completed file is empty " + instant.toString()); + } + return commitsToRollback; } List rollbackedCommits = new LinkedList<>(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java index ab5b5f6b4db82..b12869aae0984 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java @@ -24,12 +24,17 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; /** * {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table. @@ -65,16 +70,48 @@ protected FileStatus[] listPartition(Path partitionPath) throws IOException { return tableMetadata.getAllFilesInPartition(partitionPath); } + @Override + protected List getAllPartitionPaths() throws IOException { + return tableMetadata.getAllPartitionPaths(); + } + + @Override + protected Map, FileStatus[]> listPartitions(List> partitionPathList) throws IOException { + Map> absoluteToPairMap = partitionPathList.stream() + .collect(Collectors.toMap( + pair -> pair.getRight().toString(), + Function.identity() + )); + return tableMetadata.getAllFilesInPartitions( + partitionPathList.stream().map(pair -> pair.getRight().toString()).collect(Collectors.toList())) + .entrySet().stream().collect(Collectors.toMap( + entry -> absoluteToPairMap.get(entry.getKey()), + Map.Entry::getValue + )); + } + @Override public void reset() { - super.reset(); - tableMetadata.reset(); + try { + writeLock.lock(); + clear(); + // Initialize with new Hoodie timeline. + init(metaClient, getTimeline()); + tableMetadata.reset(); + } finally { + writeLock.unlock(); + } } @Override public void sync() { - super.sync(); - tableMetadata.reset(); + try { + writeLock.lock(); + maySyncIncrementally(); + tableMetadata.reset(); + } finally { + writeLock.unlock(); + } } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 8e42b7c6eaeff..81578ba26255e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -232,8 +232,11 @@ public HoodieMetadataPayload(Option recordOpt) { columnStatMetadata = HoodieMetadataColumnStats.newBuilder(METADATA_COLUMN_STATS_BUILDER_STUB.get()) .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME)) .setColumnName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_COLUMN_NAME)) - .setMinValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)) - .setMaxValue(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)) + // AVRO-2377 1.9.2 Modified the type of org.apache.avro.Schema#FIELD_RESERVED to Collections.unmodifiableSet. + // This causes Kryo to fail when deserializing a GenericRecord, See HUDI-5484. + // We should avoid using GenericRecord and convert GenericRecord into a serializable type. + .setMinValue(wrapStatisticValue(unwrapStatisticValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)))) + .setMaxValue(wrapStatisticValue(unwrapStatisticValueWrapper(columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)))) .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) .setNullCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_NULL_COUNT)) .setTotalSize((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_TOTAL_SIZE)) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/config/HoodieTestFakeConfig.java b/hudi-common/src/test/java/org/apache/hudi/common/config/HoodieTestFakeConfig.java new file mode 100644 index 0000000000000..95e213268a3e5 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/config/HoodieTestFakeConfig.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +import java.util.Properties; + +/** + * A config class extending {@link HoodieConfig} for testing only. + */ +public class HoodieTestFakeConfig extends HoodieConfig { + + public static ConfigProperty FAKE_STRING_CONFIG = TestConfigProperty.FAKE_STRING_CONFIG; + public static ConfigProperty FAKE_INTEGER_CONFIG = TestConfigProperty.FAKE_INTEGER_CONFIG; + public static ConfigProperty FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER + = TestConfigProperty.FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER; + public static ConfigProperty FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY + = TestConfigProperty.FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY; + + private HoodieTestFakeConfig() { + super(); + } + + private HoodieTestFakeConfig(Properties props) { + super(props); + } + + public String getFakeString() { + return getString(FAKE_STRING_CONFIG); + } + + public int getFakeInteger() { + return getInt(FAKE_INTEGER_CONFIG); + } + + public String getFakeStringNoDefaultWithInfer() { + return getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER); + } + + public String getFakeStringNoDefaultWithInferEmpty() { + return getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY); + } + + public static HoodieTestFakeConfig.Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private final HoodieTestFakeConfig config = new HoodieTestFakeConfig(); + + public Builder withFakeString(String value) { + config.setValue(FAKE_STRING_CONFIG, value); + return this; + } + + public Builder withFakeInteger(int value) { + config.setValue(FAKE_INTEGER_CONFIG, String.valueOf(value)); + return this; + } + + public Builder withFakeStringNoDefaultWithInfer(String value) { + config.setValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER, value); + return this; + } + + public Builder withFakeStringNoDefaultWithInferEmpty(String value) { + config.setValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY, value); + return this; + } + + public HoodieTestFakeConfig build() { + setDefaults(); + return new HoodieTestFakeConfig(config.getProps()); + } + + private void setDefaults() { + config.setDefaults(HoodieTestFakeConfig.class.getName()); + } + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java index 6cbb9bd48e496..19b27f4d3c3e2 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java @@ -56,6 +56,29 @@ public class TestConfigProperty extends HoodieConfig { }) .withDocumentation("Fake config only for testing"); + public static ConfigProperty FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER = ConfigProperty + .key("test.fake.string.config.no_default_with_infer") + .noDefaultValue() + .withInferFunction(p -> { + if (p.getStringOrDefault(FAKE_STRING_CONFIG).equals("value1")) { + return Option.of("value2"); + } + return Option.of("value3"); + }) + .withDocumentation("Fake config with infer function and without default only for testing"); + + public static ConfigProperty FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY = ConfigProperty + .key("test.fake.string.config.no_default_with_infer_empty") + .noDefaultValue() + .withInferFunction(p -> { + if (p.getStringOrDefault(FAKE_STRING_CONFIG).equals("value1")) { + return Option.of("value10"); + } + return Option.empty(); + }) + .withDocumentation("Fake config with infer function that ca return empty value " + + "and without default only for testing"); + @Test public void testGetTypedValue() { HoodieConfig hoodieConfig = new HoodieConfig(); @@ -108,11 +131,30 @@ public void testInference() { hoodieConfig2.setValue(FAKE_STRING_CONFIG, "5"); hoodieConfig2.setDefaultValue(FAKE_INTEGER_CONFIG); assertEquals(100, hoodieConfig2.getInt(FAKE_INTEGER_CONFIG)); + + HoodieConfig hoodieConfig3 = new HoodieConfig(); + hoodieConfig3.setValue(FAKE_STRING_CONFIG, "value1"); + hoodieConfig3.setDefaultValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER); + hoodieConfig3.setDefaultValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY); + assertEquals("value2", hoodieConfig3.getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER)); + assertEquals("value10", hoodieConfig3.getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY)); + + HoodieConfig hoodieConfig4 = new HoodieConfig(); + hoodieConfig4.setValue(FAKE_STRING_CONFIG, "other"); + hoodieConfig4.setDefaultValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER); + assertEquals("value3", hoodieConfig4.getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER)); + assertEquals(null, hoodieConfig4.getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY)); + + HoodieConfig hoodieConfig5 = new HoodieConfig(); + hoodieConfig5.setValue(FAKE_STRING_CONFIG, "other"); + hoodieConfig5.setValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER, "value4"); + hoodieConfig5.setDefaultValue(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER); + assertEquals("value4", hoodieConfig5.getString(FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER)); } @Test public void testSetDefaults() { setDefaults(this.getClass().getName()); - assertEquals(3, getProps().size()); + assertEquals(4, getProps().size()); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/config/TestHoodieConfig.java b/hudi-common/src/test/java/org/apache/hudi/common/config/TestHoodieConfig.java new file mode 100644 index 0000000000000..58e21d202e968 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/config/TestHoodieConfig.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Tests {@link HoodieConfig}. + */ +public class TestHoodieConfig { + @Test + public void testHoodieConfig() { + // Case 1: defaults and infer function are used + HoodieTestFakeConfig config1 = HoodieTestFakeConfig.newBuilder().build(); + assertEquals("1", config1.getFakeString()); + assertEquals(0, config1.getFakeInteger()); + assertEquals("value3", config1.getFakeStringNoDefaultWithInfer()); + assertEquals(null, config1.getFakeStringNoDefaultWithInferEmpty()); + + // Case 2: FAKE_STRING_CONFIG is set. FAKE_INTEGER_CONFIG, + // FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER, and + // FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY are inferred + HoodieTestFakeConfig config2 = HoodieTestFakeConfig.newBuilder() + .withFakeString("value1").build(); + assertEquals("value1", config2.getFakeString()); + assertEquals(0, config2.getFakeInteger()); + assertEquals("value2", config2.getFakeStringNoDefaultWithInfer()); + assertEquals("value10", config2.getFakeStringNoDefaultWithInferEmpty()); + + // Case 3: FAKE_STRING_CONFIG is set to a different value. FAKE_INTEGER_CONFIG, + // FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER, and + // FAKE_STRING_CONFIG_NO_DEFAULT_WITH_INFER_EMPTY are inferred + HoodieTestFakeConfig config3 = HoodieTestFakeConfig.newBuilder() + .withFakeString("5").build(); + assertEquals("5", config3.getFakeString()); + assertEquals(100, config3.getFakeInteger()); + assertEquals("value3", config3.getFakeStringNoDefaultWithInfer()); + assertEquals(null, config3.getFakeStringNoDefaultWithInferEmpty()); + + // Case 4: all configs are set. No default or infer function should be used + HoodieTestFakeConfig config4 = HoodieTestFakeConfig.newBuilder() + .withFakeString("5") + .withFakeInteger(200) + .withFakeStringNoDefaultWithInfer("xyz") + .withFakeStringNoDefaultWithInferEmpty("uvw").build(); + assertEquals("5", config4.getFakeString()); + assertEquals(200, config4.getFakeInteger()); + assertEquals("xyz", config4.getFakeStringNoDefaultWithInfer()); + assertEquals("uvw", config4.getFakeStringNoDefaultWithInferEmpty()); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index 0b849ebec8185..9c408a4300b14 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -38,6 +38,7 @@ import java.util.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; /** @@ -80,6 +81,15 @@ public void testProcessFilesWithExceptions() throws Exception { folders.forEach(f -> assertThrows(RuntimeException.class, () -> metaClient.getFs().mkdirs(new Path(new Path(basePath), f)))); } + @Test + public void testGetDefaultReplication() { + FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 100); + FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, ""); + HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); + assertEquals(fs.getDefaultReplication(), 3); + assertEquals(fs.getDefaultReplication(new Path(basePath)), 3); + } + /** * Fake remote FileSystem which will throw RuntimeException something like AmazonS3Exception 503. */ @@ -88,6 +98,7 @@ class FakeRemoteFileSystem extends FileSystem { private FileSystem fs; private int count = 1; private int loop; + private short defaultReplication = 3; public FakeRemoteFileSystem(FileSystem fs, int retryLoop) { this.fs = fs; @@ -206,5 +217,15 @@ public RemoteIterator listLocatedStatus(Path f) throws IOExce public Configuration getConf() { return fs.getConf(); } + + @Override + public short getDefaultReplication() { + return defaultReplication; + } + + @Override + public short getDefaultReplication(Path path) { + return defaultReplication; + } } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java index abb31ab19ea13..154c9e341acdd 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java @@ -360,7 +360,7 @@ public void testAppendNotSupported() throws IOException, URISyntaxException, Int for (int i = 0; i < 2; i++) { Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive").overBaseCommit("") + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits").overBaseCommit("") .withFs(localFs).build(); writer.appendBlock(dataBlock); writer.close(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java index 6c4d69a05b296..5fbc269b223d7 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormatAppendFailure.java @@ -108,7 +108,7 @@ public void testFailedToGetAppendStreamFromHDFSNameNode() HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header, HoodieRecord.RECORD_KEY_METADATA_FIELD); Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits") .overBaseCommit("").withFs(fs).build(); writer.appendBlock(dataBlock); @@ -139,7 +139,7 @@ public void testFailedToGetAppendStreamFromHDFSNameNode() // Opening a new Writer right now will throw IOException. The code should handle this, rollover the logfile and // return a new writer with a bumped up logVersion writer = HoodieLogFormat.newWriterBuilder().onParentPath(testPath) - .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive") + .withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits") .overBaseCommit("").withFs(fs).build(); header = new HashMap<>(); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java index 78599afc1fe16..bac841d2ef5ce 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestPostgresDebeziumAvroPayload.java @@ -18,6 +18,10 @@ package org.apache.hudi.common.model.debezium; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieDebeziumAvroPayloadException; @@ -113,6 +117,21 @@ public void testMergeWithDelete() throws IOException { validateRecord(mergedRecord, 2, Operation.UPDATE, 99L); } + @Test + public void testMergeWithDeleteUsingEmptyRecord() throws IOException { + // empty record being merged with current record. + HoodieRecord emptyRecord = new HoodieAvroRecord(new HoodieKey(), new PostgresDebeziumAvroPayload(Option.empty())); + GenericRecord existingRecord = createRecord(2, Operation.UPDATE, 99L); + Option mergedRecord = emptyRecord.getData().combineAndGetUpdateValue(existingRecord, avroSchema, new TypedProperties()); + // expect nothing to be committed to table + assertFalse(mergedRecord.isPresent()); + // Insert record being merged with empty record. + GenericRecord insertedRecord = createRecord(1, Operation.INSERT, 100L); + PostgresDebeziumAvroPayload insertPayload = new PostgresDebeziumAvroPayload(insertedRecord, 100L); + PostgresDebeziumAvroPayload combinedPayload = (PostgresDebeziumAvroPayload) insertPayload.preCombine(emptyRecord.getData(), new TypedProperties()); + assertEquals(insertPayload, combinedPayload); + } + @Test public void testMergeWithBootstrappedExistingRecords() throws IOException { GenericRecord incomingRecord = createRecord(3, Operation.UPDATE, 100L); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java index da078372b5c3b..7e3182e2871dd 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java @@ -30,6 +30,8 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -49,10 +51,26 @@ import java.util.HashMap; import java.util.List; import java.util.Map; - +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVEPOINT_ACTION; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class TestTimelineUtils extends HoodieCommonTestHarness { @@ -109,11 +127,11 @@ public void testGetPartitions() throws IOException { String olderPartition = "0"; // older partitions that is modified by all cleans for (int i = 1; i <= 5; i++) { String ts = i + ""; - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, ts, ts, 2, Collections.emptyMap()))); - HoodieInstant cleanInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, ts); + HoodieInstant cleanInstant = new HoodieInstant(true, CLEAN_ACTION, ts); activeTimeline.createNewInstant(cleanInstant); activeTimeline.saveAsComplete(cleanInstant, getCleanMetadata(olderPartition, ts)); } @@ -148,11 +166,11 @@ public void testGetPartitionsUnPartitioned() throws IOException { String partitionPath = ""; for (int i = 1; i <= 5; i++) { String ts = i + ""; - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, partitionPath, ts, 2, Collections.emptyMap()))); - HoodieInstant cleanInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, ts); + HoodieInstant cleanInstant = new HoodieInstant(true, CLEAN_ACTION, ts); activeTimeline.createNewInstant(cleanInstant); activeTimeline.saveAsComplete(cleanInstant, getCleanMetadata(partitionPath, ts)); } @@ -177,7 +195,7 @@ public void testRestoreInstants() throws Exception { String ts = i + ""; HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, ts); activeTimeline.createNewInstant(instant); - activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, HoodieTimeline.COMMIT_ACTION))); + activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, COMMIT_ACTION))); } metaClient.reloadActiveTimeline(); @@ -200,12 +218,12 @@ public void testGetExtraMetadata() throws Exception { assertFalse(TimelineUtils.getExtraMetadataFromLatest(metaClient, extraMetadataKey).isPresent()); String ts = "0"; - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, ts, ts, 2, Collections.emptyMap()))); ts = "1"; - instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, ts); + instant = new HoodieInstant(true, COMMIT_ACTION, ts); activeTimeline.createNewInstant(instant); Map extraMetadata = new HashMap<>(); extraMetadata.put(extraMetadataKey, extraMetadataValue1); @@ -240,7 +258,157 @@ public void testGetExtraMetadata() throws Exception { assertTrue(extraMetadataEntries.get("2").isPresent()); assertEquals(newValueForMetadata, extraMetadataEntries.get("2").get()); } - + + @Test + public void testGetCommitsTimelineAfter() throws IOException { + // Should only load active timeline + String startTs = "010"; + HoodieTableMetaClient mockMetaClient = prepareMetaClient( + Arrays.asList( + new HoodieInstant(COMPLETED, COMMIT_ACTION, "009"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012")), + Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002")), + startTs + ); + verifyTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012")), + TimelineUtils.getCommitsTimelineAfter(mockMetaClient, startTs)); + verify(mockMetaClient, never()).getArchivedTimeline(any()); + + // Should load both archived and active timeline + startTs = "001"; + mockMetaClient = prepareMetaClient( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "009"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012")), + Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002")), + startTs + ); + verifyTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012")), + TimelineUtils.getCommitsTimelineAfter(mockMetaClient, startTs)); + verify(mockMetaClient, times(1)).getArchivedTimeline(any()); + } + + private HoodieTableMetaClient prepareMetaClient( + List activeInstants, + List archivedInstants, + String startTs + ) throws IOException { + HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); + HoodieArchivedTimeline mockArchivedTimeline = mock(HoodieArchivedTimeline.class); + when(mockMetaClient.scanHoodieInstantsFromFileSystem(any(), eq(true))) + .thenReturn(activeInstants); + HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(mockMetaClient); + when(mockMetaClient.getActiveTimeline()) + .thenReturn(activeTimeline); + when(mockMetaClient.getArchivedTimeline(any())) + .thenReturn(mockArchivedTimeline); + HoodieDefaultTimeline mergedTimeline = new HoodieDefaultTimeline( + archivedInstants.stream() + .filter(instant -> instant.getTimestamp().compareTo(startTs) >= 0), + i -> Option.empty()) + .mergeTimeline(activeTimeline); + when(mockArchivedTimeline.mergeTimeline(eq(activeTimeline))) + .thenReturn(mergedTimeline); + + return mockMetaClient; + } + + public void verifyTimeline(List expectedInstants, HoodieTimeline timeline) { + assertEquals( + expectedInstants.stream().sorted().collect(Collectors.toList()), + timeline.getInstants().sorted().collect(Collectors.toList()) + ); + } + + @Test + public void testGetEarliestInstantForMetadataArchival() throws IOException { + // Empty timeline + assertEquals( + Option.empty(), + TimelineUtils.getEarliestInstantForMetadataArchival( + prepareActiveTimeline(new ArrayList<>()), false)); + + // Earlier request clean action before commits + assertEquals( + Option.of(new HoodieInstant(REQUESTED, CLEAN_ACTION, "003")), + TimelineUtils.getEarliestInstantForMetadataArchival( + prepareActiveTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "002"), + new HoodieInstant(REQUESTED, CLEAN_ACTION, "003"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011"))), false)); + + // No inflight instants + assertEquals( + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010")), + TimelineUtils.getEarliestInstantForMetadataArchival( + prepareActiveTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "002"), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "003"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011"))), false)); + + // Rollbacks only + assertEquals( + Option.of(new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003")), + TimelineUtils.getEarliestInstantForMetadataArchival( + prepareActiveTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002"), + new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003"))), false)); + + assertEquals( + Option.empty(), + TimelineUtils.getEarliestInstantForMetadataArchival( + prepareActiveTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002"), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "003"))), false)); + + // With savepoints + HoodieActiveTimeline timeline = prepareActiveTimeline( + Arrays.asList( + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "003"), + new HoodieInstant(COMPLETED, SAVEPOINT_ACTION, "003"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"))); + assertEquals( + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "003")), + TimelineUtils.getEarliestInstantForMetadataArchival(timeline, false)); + assertEquals( + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010")), + TimelineUtils.getEarliestInstantForMetadataArchival(timeline, true)); + } + + private HoodieActiveTimeline prepareActiveTimeline( + List activeInstants) throws IOException { + HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); + when(mockMetaClient.scanHoodieInstantsFromFileSystem(any(), eq(true))) + .thenReturn(activeInstants); + return new HoodieActiveTimeline(mockMetaClient); + } + private void verifyExtraMetadataLatestValue(String extraMetadataKey, String expected, boolean includeClustering) { final Option extraLatestValue; if (includeClustering) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 02f65886013f7..5d919a4f4d96c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -298,12 +298,14 @@ protected void testInvalidLogFiles() throws Exception { String fileName2 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 1, TEST_WRITE_TOKEN); // create a dummy log file mimicing cloud stores marker files - String fileName3 = "_DUMMY_" + fileName1.substring(1, fileName1.length()); + String fileName3 = "_GCS_SYNCABLE_TEMPFILE_" + fileName1; + String fileName4 = "_DUMMY_" + fileName1.substring(1, fileName1.length()); // this file should not be deduced as a log file. Paths.get(basePath, partitionPath, fileName1).toFile().createNewFile(); Paths.get(basePath, partitionPath, fileName2).toFile().createNewFile(); Paths.get(basePath, partitionPath, fileName3).toFile().createNewFile(); + Paths.get(basePath, partitionPath, fileName4).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 8be78a3a96927..9d24e596cbc6b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -98,7 +98,12 @@ public static String markerFileName(String instantTime, String fileId, IOType io } public static String markerFileName(String instantTime, String fileId, IOType ioType, String fileExtension) { - return String.format("%s_%s_%s%s%s.%s", fileId, WRITE_TOKEN, instantTime, fileExtension, HoodieTableMetaClient.MARKER_EXTN, ioType); + return markerFileName(instantTime, fileId, ioType, fileExtension, WRITE_TOKEN); + } + + public static String markerFileName(String instantTime, String fileId, IOType ioType, String fileExtension, String writeToken) { + return String.format("%s_%s_%s%s%s.%s", fileId, writeToken, instantTime, fileExtension, + HoodieTableMetaClient.MARKER_EXTN, ioType); } private static void createMetaFile(String basePath, String instantTime, String suffix, FileSystem fs) throws IOException { @@ -353,9 +358,14 @@ public static void createLogFile(String basePath, String partitionPath, String i public static String createMarkerFile(String basePath, String partitionPath, String instantTime, String fileId, IOType ioType) throws IOException { - Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, instantTime, partitionPath); + return createMarkerFile(basePath, partitionPath, instantTime, instantTime, fileId, ioType, WRITE_TOKEN); + } + + public static String createMarkerFile(String basePath, String partitionPath, String commitInstant, + String instantTime, String fileId, IOType ioType, String writeToken) throws IOException { + Path parentPath = Paths.get(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitInstant, partitionPath); Files.createDirectories(parentPath); - Path markerFilePath = parentPath.resolve(markerFileName(instantTime, fileId, ioType)); + Path markerFilePath = parentPath.resolve(markerFileName(instantTime, fileId, ioType, BASE_FILE_EXTENSION, writeToken)); if (Files.notExists(markerFilePath)) { Files.createFile(markerFilePath); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 8614060126dfa..c915e8368eedd 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -72,6 +72,7 @@ import java.util.Random; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -513,8 +514,14 @@ public List generateInsertsForPartition(String instantTime, Intege } public Stream generateInsertsStream(String commitTime, Integer n, boolean isFlattened, String schemaStr, boolean containsAllPartitions) { + AtomicInteger partitionIndex = new AtomicInteger(0); return generateInsertsStream(commitTime, n, isFlattened, schemaStr, containsAllPartitions, - () -> partitionPaths[rand.nextInt(partitionPaths.length)], + () -> { + // round robin to ensure we generate inserts for all partition paths + String partitionToUse = partitionPaths[partitionIndex.get()]; + partitionIndex.set((partitionIndex.get() + 1) % partitionPaths.length); + return partitionToUse; + }, () -> genPseudoRandomUUID(rand).toString()); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java index a5d45d1184f9b..edd877a28fdbb 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java @@ -18,17 +18,21 @@ package org.apache.hudi.common.util; +import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -43,6 +47,7 @@ import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; /** * Tests for {@link ClusteringUtils}. @@ -114,6 +119,70 @@ public void testClusteringPlanInflight() throws Exception { assertEquals(requestedClusteringPlan, inflightClusteringPlan); } + @Test + public void testGetOldestInstantToRetainForClustering() throws IOException { + String partitionPath1 = "partition1"; + List fileIds1 = new ArrayList<>(); + fileIds1.add(UUID.randomUUID().toString()); + String clusterTime1 = "1"; + HoodieInstant requestedInstant1 = createRequestedReplaceInstant(partitionPath1, clusterTime1, fileIds1); + HoodieInstant inflightInstant1 = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant1, Option.empty()); + HoodieInstant completedInstant1 = metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant1, Option.empty()); + List fileIds2 = new ArrayList<>(); + fileIds2.add(UUID.randomUUID().toString()); + fileIds2.add(UUID.randomUUID().toString()); + String clusterTime2 = "2"; + HoodieInstant requestedInstant2 = createRequestedReplaceInstant(partitionPath1, clusterTime2, fileIds2); + HoodieInstant inflightInstant2 = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant2, Option.empty()); + metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant2, Option.empty()); + List fileIds3 = new ArrayList<>(); + fileIds3.add(UUID.randomUUID().toString()); + fileIds3.add(UUID.randomUUID().toString()); + fileIds3.add(UUID.randomUUID().toString()); + String clusterTime3 = "3"; + HoodieInstant requestedInstant3 = createRequestedReplaceInstant(partitionPath1, clusterTime3, fileIds3); + HoodieInstant inflightInstant3 = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant3, Option.empty()); + HoodieInstant completedInstant3 = metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant3, Option.empty()); + metaClient.reloadActiveTimeline(); + Option actual = ClusteringUtils.getOldestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient); + assertFalse(actual.isPresent()); + // test first uncompleted clean instant is requested. + String cleanTime1 = "4"; + HoodieInstant requestedInstant4 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); + HoodieCleanerPlan cleanerPlan1 = HoodieCleanerPlan.newBuilder() + .setEarliestInstantToRetainBuilder(HoodieActionInstant.newBuilder() + .setAction(completedInstant1.getAction()) + .setTimestamp(completedInstant1.getTimestamp()) + .setState(completedInstant1.getState().name())) + .setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) + .setFilesToBeDeletedPerPartition(new HashMap<>()) + .setVersion(CleanPlanV2MigrationHandler.VERSION) + .build(); + metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant4, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan1)); + metaClient.reloadActiveTimeline(); + actual = ClusteringUtils.getOldestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient); + assertEquals(clusterTime1, actual.get().getTimestamp()); + HoodieInstant inflightInstant4 = metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant4, Option.empty()); + metaClient.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant4, Option.empty()); + // test first uncompleted clean instant is inflight. + String cleanTime2 = "5"; + HoodieInstant requestedInstant5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime2); + HoodieCleanerPlan cleanerPlan2 = HoodieCleanerPlan.newBuilder() + .setEarliestInstantToRetainBuilder(HoodieActionInstant.newBuilder() + .setAction(completedInstant3.getAction()) + .setTimestamp(completedInstant3.getTimestamp()) + .setState(completedInstant3.getState().name())) + .setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) + .setFilesToBeDeletedPerPartition(new HashMap<>()) + .setVersion(CleanPlanV2MigrationHandler.VERSION) + .build(); + metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant5, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan2)); + metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant5, Option.empty()); + metaClient.reloadActiveTimeline(); + actual = ClusteringUtils.getOldestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient); + assertEquals(clusterTime3, actual.get().getTimestamp()); + } + private void validateClusteringInstant(List fileIds, String partitionPath, String expectedInstantTime, Map fileGroupToInstantMap) { for (String fileId : fileIds) { diff --git a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java index 6126c479c6154..1f71cc9969b72 100644 --- a/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java @@ -48,12 +48,12 @@ public class TestAvroSchemaEvolutionUtils { String schemaStr = "{\"type\":\"record\",\"name\":\"newTableName\",\"fields\":[{\"name\":\"id\",\"type\":\"int\"},{\"name\":\"data\"," + "\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"preferences\",\"type\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_preferences\",\"fields\":[{\"name\":\"feature1\"," + + "{\"type\":\"record\",\"name\":\"preferences\",\"namespace\":\"newTableName\",\"fields\":[{\"name\":\"feature1\"," + "\"type\":\"boolean\"},{\"name\":\"feature2\",\"type\":[\"null\",\"boolean\"],\"default\":null}]}]," + "\"default\":null},{\"name\":\"locations\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"record\"," - + "\"name\":\"newTableName_locations\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," + + "\"name\":\"locations\",\"namespace\":\"newTableName\",\"fields\":[{\"name\":\"lat\",\"type\":\"float\"},{\"name\":\"long\"," + "\"type\":\"float\"}]}}},{\"name\":\"points\",\"type\":[\"null\",{\"type\":\"array\",\"items\":[\"null\"," - + "{\"type\":\"record\",\"name\":\"newTableName_points\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," + + "{\"type\":\"record\",\"name\":\"points\",\"namespace\":\"newTableName\",\"fields\":[{\"name\":\"x\",\"type\":\"long\"}," + "{\"name\":\"y\",\"type\":\"long\"}]}]}],\"default\":null},{\"name\":\"doubles\",\"type\":{\"type\":\"array\",\"items\":\"double\"}}," + "{\"name\":\"properties\",\"type\":[\"null\",{\"type\":\"map\",\"values\":[\"null\",\"string\"]}],\"default\":null}]}"; @@ -69,10 +69,10 @@ public void testPrimitiveTypes() { LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)), LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), Schema.create(Schema.Type.STRING), - LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16)), - Schema.createFixed("fixed_12", null, null, 12), + LogicalTypes.uuid().addToSchema(Schema.createFixed("t1.fixed", null, null, 16)), + Schema.createFixed("t1.fixed", null, null, 12), Schema.create(Schema.Type.BYTES), - LogicalTypes.decimal(9, 4).addToSchema(Schema.createFixed("decimal_9_4", null, null, 4))}; + LogicalTypes.decimal(9, 4).addToSchema(Schema.createFixed("t1.fixed", null, null, 4))}; Type[] primitiveTypes = new Type[] { Types.BooleanType.get(), @@ -126,11 +126,11 @@ public void testRecordAndPrimitiveTypes() { new Schema.Field("time", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE), new Schema.Field("timestamp", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))), null, JsonProperties.NULL_VALUE), new Schema.Field("string", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.STRING)), null, JsonProperties.NULL_VALUE), - new Schema.Field("uuid", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16))), null, JsonProperties.NULL_VALUE), - new Schema.Field("fixed", AvroInternalSchemaConverter.nullableSchema(Schema.createFixed("fixed_10", null, null, 10)), null, JsonProperties.NULL_VALUE), + new Schema.Field("uuid", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.uuid().addToSchema(Schema.createFixed("t1.uuid.fixed", null, null, 16))), null, JsonProperties.NULL_VALUE), + new Schema.Field("fixed", AvroInternalSchemaConverter.nullableSchema(Schema.createFixed("t1.fixed.fixed", null, null, 10)), null, JsonProperties.NULL_VALUE), new Schema.Field("binary", AvroInternalSchemaConverter.nullableSchema(Schema.create(Schema.Type.BYTES)), null, JsonProperties.NULL_VALUE), new Schema.Field("decimal", AvroInternalSchemaConverter.nullableSchema(LogicalTypes.decimal(10, 2) - .addToSchema(Schema.createFixed("decimal_10_2", null, null, 5))), null, JsonProperties.NULL_VALUE)); + .addToSchema(Schema.createFixed("t1.decimal.fixed", null, null, 5))), null, JsonProperties.NULL_VALUE)); Schema convertedSchema = AvroInternalSchemaConverter.convert(record, "t1"); Assertions.assertEquals(convertedSchema, schema); Types.RecordType convertedRecord = AvroInternalSchemaConverter.convert(schema).getRecord(); @@ -160,7 +160,7 @@ public void testArrayType() { public void testComplexConvert() { Schema schema = new Schema.Parser().parse(schemaStr); - InternalSchema internalSchema = new InternalSchema(Types.Field.get(0, false, "id", Types.IntType.get()), + Types.RecordType recordType = Types.RecordType.get(Types.Field.get(0, false, "id", Types.IntType.get()), Types.Field.get(1, true, "data", Types.StringType.get()), Types.Field.get(2, true, "preferences", Types.RecordType.get(Types.Field.get(7, false, "feature1", @@ -169,9 +169,10 @@ public void testComplexConvert() { Types.RecordType.get(Types.Field.get(11, false, "lat", Types.FloatType.get()), Types.Field.get(12, false, "long", Types.FloatType.get())), false)), Types.Field.get(4, true, "points", Types.ArrayType.get(13, true, Types.RecordType.get(Types.Field.get(14, false, "x", Types.LongType.get()), Types.Field.get(15, false, "y", Types.LongType.get())))), - Types.Field.get(5, false,"doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), + Types.Field.get(5, false, "doubles", Types.ArrayType.get(16, false, Types.DoubleType.get())), Types.Field.get(6, true, "properties", Types.MapType.get(17, 18, Types.StringType.get(), Types.StringType.get())) ); + InternalSchema internalSchema = new InternalSchema(recordType); Type convertRecord = AvroInternalSchemaConverter.convert(schema).getRecord(); Assertions.assertEquals(convertRecord, internalSchema.getRecord()); @@ -285,7 +286,7 @@ public void testReWriteRecordWithTypeChanged() { .updateColumnType("col51", Types.DecimalType.get(18, 9)) .updateColumnType("col6", Types.StringType.get()); InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); - Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName()); + Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getFullName()); GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, Collections.emptyMap()); Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true); @@ -308,14 +309,14 @@ public void testReWriteNestRecord() { avroRecord.put("id", 2); avroRecord.put("data", "xs"); // fill record type - GenericData.Record preferencesRecord = new GenericData.Record(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences")); + GenericData.Record preferencesRecord = new GenericData.Record(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1.preferences")); preferencesRecord.put("feature1", false); preferencesRecord.put("feature2", true); - Assertions.assertEquals(GenericData.get().validate(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1_preferences"), preferencesRecord), true); + Assertions.assertEquals(GenericData.get().validate(AvroInternalSchemaConverter.convert(record.fieldType("preferences"), "test1.preferences"), preferencesRecord), true); avroRecord.put("preferences", preferencesRecord); // fill mapType Map locations = new HashMap<>(); - Schema mapSchema = AvroInternalSchemaConverter.convert(((Types.MapType)record.field("locations").type()).valueType(), "test1_locations"); + Schema mapSchema = AvroInternalSchemaConverter.convert(((Types.MapType)record.field("locations").type()).valueType(), "test1.locations"); GenericData.Record locationsValue = new GenericData.Record(mapSchema); locationsValue.put("lat", 1.2f); locationsValue.put("long", 1.4f); @@ -332,7 +333,7 @@ public void testReWriteNestRecord() { avroRecord.put("doubles", doubles); // do check - Assertions.assertEquals(GenericData.get().validate(schema, avroRecord), true); + Assertions.assertTrue(GenericData.get().validate(schema, avroRecord)); // create newSchema Types.RecordType newRecord = Types.RecordType.get( Types.Field.get(0, false, "id", Types.IntType.get()), @@ -366,7 +367,7 @@ public void testReWriteNestRecord() { .renameColumn("preferences.feature2", "f2") .renameColumn("locations.value.lat", "lt"); InternalSchema internalSchemaRename = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange); - Schema avroSchemaRename = AvroInternalSchemaConverter.convert(internalSchemaRename, schema.getName()); + Schema avroSchemaRename = AvroInternalSchemaConverter.convert(internalSchemaRename, schema.getFullName()); Map renameCols = InternalSchemaUtils.collectRenameCols(internalSchema, internalSchemaRename); GenericRecord avroRecordRename = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, avroSchemaRename, renameCols); // test the correctly of rewrite @@ -390,7 +391,7 @@ public void testEvolutionSchemaFromNewAvroSchema() { Types.Field.get(12, false, "long", Types.FloatType.get())), false) ) ); - InternalSchema oldSchema = new InternalSchema(oldRecord.fields()); + InternalSchema oldSchema = new InternalSchema(oldRecord); Types.RecordType evolvedRecord = Types.RecordType.get( Types.Field.get(0, false, "id", Types.IntType.get()), Types.Field.get(1, true, "data", Types.StringType.get()), diff --git a/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java b/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.java new file mode 100644 index 0000000000000..ccd2aa04cee88 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/metadata/HoodieBackedTestDelayedTableMetadata.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.metadata; + +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.engine.HoodieEngineContext; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Table metadata provided by an internal DFS backed Hudi metadata table, + * with an intentional delay in `reset()` to test concurrent reads and writes. + */ +public class HoodieBackedTestDelayedTableMetadata extends HoodieBackedTableMetadata { + private static final Logger LOG = LogManager.getLogger(HoodieBackedTestDelayedTableMetadata.class); + + public HoodieBackedTestDelayedTableMetadata(HoodieEngineContext engineContext, + HoodieMetadataConfig metadataConfig, + String datasetBasePath, + String spillableMapDirectory, + boolean reuse) { + super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory, reuse); + } + + @Override + public void reset() { + LOG.info("Sleeping for 5 seconds in reset() to simulate processing ..."); + try { + Thread.sleep(5000); + } catch (InterruptedException e) { + LOG.warn("Sleep is interrupted", e); + } + LOG.info("Sleep in reset() is finished."); + super.reset(); + } +} diff --git a/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/source/ContinuousFileSource.java b/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/source/ContinuousFileSource.java index f1696f332824f..baecf43003f35 100644 --- a/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/source/ContinuousFileSource.java +++ b/hudi-examples/hudi-examples-flink/src/main/java/org/apache/hudi/examples/quickstart/source/ContinuousFileSource.java @@ -19,12 +19,11 @@ package org.apache.hudi.examples.quickstart.source; import org.apache.hudi.adapter.DataStreamScanProviderAdapter; +import org.apache.hudi.util.JsonDeserializationFunction; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -37,7 +36,6 @@ import org.apache.flink.table.types.logical.RowType; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; @@ -49,7 +47,7 @@ * A continuous file source that can trigger checkpoints continuously. * *

It loads the data in the specified file and split the data into number of checkpoints batches. - * Say, if you want 4 checkpoints and there are 8 records in the file, the emit strategy is: + * Say, if you want 4 checkpoints and there are 8 records in the file, the emission strategy is: * *

  *   | 2 records | 2 records | 2 records | 2 records |
@@ -57,6 +55,8 @@
  * 
* *

If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source. + * + *

NOTE: this class is represented twice: in test utils and in quickstart. Don't forget to update both files. */ public class ContinuousFileSource implements ScanTableSource { @@ -85,18 +85,10 @@ public boolean isBounded() { @Override public DataStream produceDataStream(StreamExecutionEnvironment execEnv) { final RowType rowType = (RowType) tableSchema.toSourceRowDataType().getLogicalType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601); - return execEnv.addSource(new BoundedSourceFunction(path, conf.getInteger(CHECKPOINTS))) .name("continuous_file_source") .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)), - InternalTypeInfo.of(rowType)); + .map(JsonDeserializationFunction.getInstance(rowType), InternalTypeInfo.of(rowType)); } }; } diff --git a/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java b/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java index c23db7f8e7106..486e7f180d64c 100644 --- a/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java +++ b/hudi-examples/hudi-examples-spark/src/test/java/org/apache/hudi/examples/quickstart/TestHoodieSparkQuickstart.java @@ -21,8 +21,7 @@ import org.apache.hudi.client.SparkRDDReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.examples.common.HoodieExampleDataGenerator; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.testutils.providers.SparkProvider; import org.apache.spark.SparkConf; @@ -37,15 +36,7 @@ import java.io.File; import java.nio.file.Paths; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.delete; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.deleteByPartition; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.incrementalQuery; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.insertData; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.insertOverwriteData; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.pointInTimeQuery; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.queryData; import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.runQuickstart; -import static org.apache.hudi.examples.quickstart.HoodieSparkQuickstart.updateData; public class TestHoodieSparkQuickstart implements SparkProvider { protected static HoodieSparkEngineContext context; @@ -66,6 +57,11 @@ public SparkSession spark() { return spark; } + @Override + public SparkConf conf() { + return conf(SparkClientFunctionalTestHarness.getSparkSqlConf()); + } + @Override public SQLContext sqlContext() { return sqlContext; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java index bcd4f407d6240..bef63089fd99a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -83,6 +84,12 @@ private FlinkOptions() { // Common Options // ------------------------------------------------------------------------ + public static final ConfigOption DATABASE_NAME = ConfigOptions + .key(HoodieTableConfig.DATABASE_NAME.key()) + .stringType() + .noDefaultValue() + .withDescription("Database name to register to Hive metastore"); + public static final ConfigOption TABLE_NAME = ConfigOptions .key(HoodieWriteConfig.TBL_NAME.key()) .stringType() @@ -331,9 +338,9 @@ private FlinkOptions() { .key("write.ignore.failed") .booleanType() .defaultValue(false) - .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch.\n" - + "By default false. Turning this on, could hide the write status errors while the spark checkpoint moves ahead. \n" - + " So, would recommend users to use this with caution."); + .withDescription("Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch. \n" + + "By default false. Turning this on, could hide the write status errors while the flink checkpoint moves ahead. \n" + + "So, would recommend users to use this with caution."); public static final ConfigOption RECORD_KEY_FIELD = ConfigOptions .key(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()) @@ -666,7 +673,9 @@ private FlinkOptions() { + "RECENT_DAYS: keep a continuous range of partitions, worked together with configs '" + DAYBASED_LOOKBACK_PARTITIONS.key() + "' and '" + PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST.key() + "." + "SELECTED_PARTITIONS: keep partitions that are in the specified range ['" + PARTITION_FILTER_BEGIN_PARTITION.key() + "', '" - + PARTITION_FILTER_END_PARTITION.key() + "']."); + + PARTITION_FILTER_END_PARTITION.key() + "']." + + "DAY_ROLLING: clustering partitions on a rolling basis by the hour to avoid clustering all partitions each time, " + + "which strategy sorts the partitions asc and chooses the partition of which index is divided by 24 and the remainder is equal to the current hour."); public static final ConfigOption CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES = ConfigOptions .key("clustering.plan.strategy.target.file.max.bytes") diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java index cbd942616fead..02467ee7cab53 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/OptionsResolver.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.format.FilePathUtils; import org.apache.flink.configuration.Configuration; @@ -196,4 +197,12 @@ public static boolean isSpecificStartCommit(Configuration conf) { public static boolean hasNoSpecificReadCommits(Configuration conf) { return !conf.contains(FlinkOptions.READ_START_COMMIT) && !conf.contains(FlinkOptions.READ_END_COMMIT); } + + /** + * Returns whether consistent value will be generated for a logical timestamp type column. + */ + public static boolean isConsistentLogicalTimestampEnabled(Configuration conf) { + return conf.getBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), + Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java index 2748af5290646..0bc22140b5c66 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java @@ -46,10 +46,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Random; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -139,7 +141,6 @@ public void processElement(I value, ProcessFunction.Context ctx, Coll @Override public void close() { if (this.writeClient != null) { - this.writeClient.cleanHandlesGracefully(); this.writeClient.close(); } } @@ -390,10 +391,9 @@ protected void bufferRecord(HoodieRecord value) { } } else if (flushBuffer) { // find the max size bucket and flush it out - List sortedBuckets = this.buckets.values().stream() - .sorted((b1, b2) -> Long.compare(b2.detector.totalSize, b1.detector.totalSize)) - .collect(Collectors.toList()); - final DataBucket bucketToFlush = sortedBuckets.get(0); + DataBucket bucketToFlush = this.buckets.values().stream() + .max(Comparator.comparingLong(b -> b.detector.totalSize)) + .orElseThrow(NoSuchElementException::new); if (flushBucket(bucketToFlush)) { this.tracer.countDown(bucketToFlush.detector.totalSize); bucketToFlush.reset(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 17b789e2f0dcc..953f9c276add1 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -32,6 +32,7 @@ import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.sink.event.CommitAckEvent; import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.meta.CkpMetadata; @@ -333,7 +334,9 @@ private void syncHive() { * Sync hoodie table metadata to Hive metastore. */ public void doSyncHive() { - hiveSyncContext.hiveSyncTool().syncHoodieTable(); + try (HiveSyncTool syncTool = hiveSyncContext.hiveSyncTool()) { + syncTool.syncHoodieTable(); + } } private static void initMetadataTable(HoodieFlinkWriteClient writeClient) { @@ -392,7 +395,7 @@ private void initInstant(String instant) { HoodieTimeline completedTimeline = StreamerUtil.createMetaClient(conf).getActiveTimeline().filterCompletedInstants(); executor.execute(() -> { - if (instant.equals("") || completedTimeline.containsInstant(instant)) { + if (instant.equals(WriteMetadataEvent.BOOTSTRAP_INSTANT) || completedTimeline.containsInstant(instant)) { // the last instant committed successfully reset(); } else { @@ -410,7 +413,11 @@ private void handleBootstrapEvent(WriteMetadataEvent event) { this.eventBuffer[event.getTaskID()] = event; if (Arrays.stream(eventBuffer).allMatch(evt -> evt != null && evt.isBootstrap())) { // start to initialize the instant. - initInstant(event.getInstantTime()); + final String instant = Arrays.stream(eventBuffer) + .filter(evt -> evt.getWriteStatuses().size() > 0) + .findFirst().map(WriteMetadataEvent::getInstantTime) + .orElse(WriteMetadataEvent.BOOTSTRAP_INSTANT); + initInstant(instant); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java index 1ccfe91dbc0a9..1814c47362f2e 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/BucketStreamWriteFunction.java @@ -111,7 +111,7 @@ public void processElement(I i, ProcessFunction.Context context, Coll bootstrapIndexIfNeed(partition); Map bucketToFileId = bucketIndex.computeIfAbsent(partition, p -> new HashMap<>()); final int bucketNum = BucketIdentifier.getBucketId(hoodieKey, indexKeyFields, this.bucketNum); - final String bucketId = partition + bucketNum; + final String bucketId = partition + "/" + bucketNum; if (incBucketIndex.contains(bucketId)) { location = new HoodieRecordLocation("I", bucketToFileId.get(bucketNum)); @@ -134,8 +134,9 @@ public void processElement(I i, ProcessFunction.Context context, Coll * (partition + curBucket) % numPartitions == this taskID belongs to this task. */ public boolean isBucketToLoad(int bucketNumber, String partition) { - int globalHash = ((partition + bucketNumber).hashCode()) & Integer.MAX_VALUE; - return BucketIdentifier.mod(globalHash, parallelism) == taskID; + final int partitionIndex = (partition.hashCode() & Integer.MAX_VALUE) % parallelism; + int globalIndex = partitionIndex + bucketNumber; + return BucketIdentifier.mod(globalIndex, parallelism) == taskID; } /** @@ -151,17 +152,18 @@ private void bootstrapIndexIfNeed(String partition) { // Load existing fileID belongs to this task Map bucketToFileIDMap = new HashMap<>(); - this.writeClient.getHoodieTable().getFileSystemView().getAllFileGroups(partition).forEach(fileGroup -> { - String fileID = fileGroup.getFileGroupId().getFileId(); - int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileID); + this.writeClient.getHoodieTable().getHoodieView().getLatestFileSlices(partition).forEach(fileSlice -> { + String fileId = fileSlice.getFileId(); + int bucketNumber = BucketIdentifier.bucketIdFromFileId(fileId); if (isBucketToLoad(bucketNumber, partition)) { - LOG.info(String.format("Should load this partition bucket %s with fileID %s", bucketNumber, fileID)); + LOG.info(String.format("Should load this partition bucket %s with fileId %s", bucketNumber, fileId)); + // Validate that one bucketId has only ONE fileId if (bucketToFileIDMap.containsKey(bucketNumber)) { - throw new RuntimeException(String.format("Duplicate fileID %s from bucket %s of partition %s found " - + "during the BucketStreamWriteFunction index bootstrap.", fileID, bucketNumber, partition)); + throw new RuntimeException(String.format("Duplicate fileId %s from bucket %s of partition %s found " + + "during the BucketStreamWriteFunction index bootstrap.", fileId, bucketNumber, partition)); } else { - LOG.info(String.format("Adding fileID %s to the bucket %s of partition %s.", fileID, bucketNumber, partition)); - bucketToFileIDMap.put(bucketNumber, fileID); + LOG.info(String.format("Adding fileId %s to the bucket %s of partition %s.", fileId, bucketNumber, partition)); + bucketToFileIDMap.put(bucketNumber, fileId); } } }); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java index 9fbdbdd8e1afc..ec4e56c2d1d06 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java @@ -127,7 +127,6 @@ public void processElement(I value, Context ctx, Collector out) throws I @Override public void close() { if (this.writeClient != null) { - this.writeClient.cleanHandlesGracefully(); this.writeClient.close(); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java index a2414abc3de21..d5cc56db5e7f5 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieKeyException; import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator; import org.apache.hudi.util.RowDataProjection; @@ -64,6 +65,7 @@ public class RowDataKeyGen implements Serializable { private final boolean hiveStylePartitioning; private final boolean encodePartitionPath; + private final boolean consistentLogicalTimestampEnabled; private final Option keyGenOpt; @@ -82,6 +84,7 @@ private RowDataKeyGen( RowType rowType, boolean hiveStylePartitioning, boolean encodePartitionPath, + boolean consistentLogicalTimestampEnabled, Option keyGenOpt) { this.recordKeyFields = recordKeys.split(","); this.partitionPathFields = partitionFields.split(","); @@ -90,6 +93,7 @@ private RowDataKeyGen( this.hiveStylePartitioning = hiveStylePartitioning; this.encodePartitionPath = encodePartitionPath; + this.consistentLogicalTimestampEnabled = consistentLogicalTimestampEnabled; if (this.recordKeyFields.length == 1) { // efficient code path this.simpleRecordKey = true; @@ -124,9 +128,10 @@ public static RowDataKeyGen instance(Configuration conf, RowType rowType) { throw new HoodieKeyException("Initialize TimestampBasedAvroKeyGenerator error", e); } } + boolean consistentLogicalTimestampEnabled = OptionsResolver.isConsistentLogicalTimestampEnabled(conf); return new RowDataKeyGen(conf.getString(FlinkOptions.RECORD_KEY_FIELD), conf.getString(FlinkOptions.PARTITION_PATH_FIELD), rowType, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING), conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING), - keyGeneratorOpt); + consistentLogicalTimestampEnabled,keyGeneratorOpt); } public HoodieKey getHoodieKey(RowData rowData) { @@ -135,10 +140,10 @@ public HoodieKey getHoodieKey(RowData rowData) { public String getRecordKey(RowData rowData) { if (this.simpleRecordKey) { - return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0]); + return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0], consistentLogicalTimestampEnabled); } else { Object[] keyValues = this.recordKeyProjection.projectAsValues(rowData); - return getRecordKey(keyValues, this.recordKeyFields); + return getRecordKey(keyValues, this.recordKeyFields, consistentLogicalTimestampEnabled); } } @@ -155,12 +160,14 @@ public String getPartitionPath(RowData rowData) { } // reference: org.apache.hudi.keygen.KeyGenUtils.getRecordPartitionPath - private static String getRecordKey(Object[] keyValues, String[] keyFields) { + private static String getRecordKey(Object[] keyValues, String[] keyFields, boolean consistentLogicalTimestampEnabled) { boolean keyIsNullEmpty = true; StringBuilder recordKey = new StringBuilder(); for (int i = 0; i < keyValues.length; i++) { String recordKeyField = keyFields[i]; - String recordKeyValue = StringUtils.objToString(keyValues[i]); + Object value = keyValues[i]; + value = getTimestampValue(consistentLogicalTimestampEnabled, value); + String recordKeyValue = StringUtils.objToString(value); if (recordKeyValue == null) { recordKey.append(recordKeyField).append(":").append(NULL_RECORDKEY_PLACEHOLDER).append(","); } else if (recordKeyValue.isEmpty()) { @@ -178,6 +185,16 @@ private static String getRecordKey(Object[] keyValues, String[] keyFields) { return recordKey.toString(); } + private static Object getTimestampValue(boolean consistentLogicalTimestampEnabled, Object value) { + if (!consistentLogicalTimestampEnabled) { + if (value instanceof TimestampData) { + TimestampData timestampData = (TimestampData) value; + value = timestampData.toTimestamp().toInstant().toEpochMilli(); + } + } + return value; + } + // reference: org.apache.hudi.keygen.KeyGenUtils.getRecordPartitionPath private static String getRecordPartitionPath( Object[] partValues, @@ -204,7 +221,8 @@ private static String getRecordPartitionPath( } // reference: org.apache.hudi.keygen.KeyGenUtils.getRecordKey - public static String getRecordKey(Object recordKeyValue, String recordKeyField) { + public static String getRecordKey(Object recordKeyValue, String recordKeyField,boolean consistentLogicalTimestampEnabled) { + recordKeyValue = getTimestampValue(consistentLogicalTimestampEnabled, recordKeyValue); String recordKey = StringUtils.objToString(recordKeyValue); if (recordKey == null || recordKey.isEmpty()) { throw new HoodieKeyException("recordKey value: \"" + recordKey + "\" for field: \"" + recordKeyField + "\" cannot be null or empty."); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java index 5a46dcf8f3360..7bf3ccb8ce616 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java @@ -147,6 +147,17 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, List< .flatMap(Collection::stream) .collect(Collectors.toList()); + long numErrorRecords = statuses.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L); + + if (numErrorRecords > 0 && !this.conf.getBoolean(FlinkOptions.IGNORE_FAILED)) { + // handle failure case + LOG.error("Got {} error records during clustering of instant {},\n" + + "option '{}' is configured as false," + + "rolls back the clustering", numErrorRecords, instant, FlinkOptions.IGNORE_FAILED.key()); + CompactionUtil.rollbackCompaction(table, instant); + return; + } + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); writeMetadata.setWriteStatuses(statuses); writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList())); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java index e7bde41ca8b0a..fe7a621abd354 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java @@ -60,6 +60,7 @@ import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; @@ -168,6 +169,11 @@ public void processWatermark(Watermark mark) { // no need to propagate the watermark } + @Override + public void processLatencyMarker(LatencyMarker latencyMarker) { + // no need to propagate the latencyMarker + } + @Override public void processElement(StreamRecord element) throws Exception { ClusteringPlanEvent event = element.getValue(); @@ -191,7 +197,6 @@ public void close() throws Exception { this.executor.close(); } if (this.writeClient != null) { - this.writeClient.cleanHandlesGracefully(); this.writeClient.close(); this.writeClient = null; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java index 1942b1ce29e71..b451c36418647 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -76,12 +76,10 @@ public HoodieFlinkClusteringJob(AsyncClusteringService service) { } public static void main(String[] args) throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - FlinkClusteringConfig cfg = getFlinkClusteringConfig(args); Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); - AsyncClusteringService service = new AsyncClusteringService(cfg, conf, env); + AsyncClusteringService service = new AsyncClusteringService(cfg, conf); new HoodieFlinkClusteringJob(service).start(cfg.serviceMode); } @@ -165,20 +163,14 @@ public static class AsyncClusteringService extends HoodieAsyncTableService { */ private final HoodieFlinkTable table; - /** - * Flink Execution Environment. - */ - private final StreamExecutionEnvironment env; - /** * Executor Service. */ private final ExecutorService executor; - public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception { + public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf) throws Exception { this.cfg = cfg; this.conf = conf; - this.env = env; this.executor = Executors.newFixedThreadPool(1); // create metaClient @@ -338,6 +330,8 @@ private void cluster() throws Exception { final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); final RowType rowType = (RowType) rowDataType.getLogicalType(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + // setup configuration long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java index f8438a4eb245c..fa4c3db86eac1 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java @@ -217,8 +217,9 @@ private void sendBootstrapEvent() { if (this.currentInstant != null) { LOG.info("Recover task[{}] for instant [{}] with attemptId [{}]", taskID, this.currentInstant, attemptId); this.currentInstant = null; + return; } - return; + // the JM may have also been rebooted, sends the bootstrap event either } this.eventGateway.sendEventToCoordinator(WriteMetadataEvent.emptyBootstrap(taskID)); LOG.info("Send bootstrap write metadata event to coordinator, task[{}].", taskID); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactOperator.java index c5a59376b5821..2db245f99f2dd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactOperator.java @@ -33,6 +33,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.runtime.operators.TableStreamOperator; import org.apache.flink.table.runtime.util.StreamRecordCollector; @@ -101,6 +102,11 @@ public void processWatermark(Watermark mark) { // no need to propagate the watermark } + @Override + public void processLatencyMarker(LatencyMarker latencyMarker) { + // no need to propagate the latencyMarker + } + @Override public void processElement(StreamRecord record) throws Exception { final CompactionPlanEvent event = record.getValue(); @@ -154,7 +160,6 @@ public void close() throws Exception { this.executor.close(); } if (null != this.writeClient) { - this.writeClient.cleanHandlesGracefully(); this.writeClient.close(); this.writeClient = null; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java index ef182241e4f17..224b981d72dc7 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSink.java @@ -157,6 +157,17 @@ private void doCommit(String instant, Collection events) .flatMap(Collection::stream) .collect(Collectors.toList()); + long numErrorRecords = statuses.stream().map(WriteStatus::getTotalErrorRecords).reduce(Long::sum).orElse(0L); + + if (numErrorRecords > 0 && !this.conf.getBoolean(FlinkOptions.IGNORE_FAILED)) { + // handle failure case + LOG.error("Got {} error records during compaction of instant {},\n" + + "option '{}' is configured as false," + + "rolls back the compaction", numErrorRecords, instant, FlinkOptions.IGNORE_FAILED.key()); + CompactionUtil.rollbackCompaction(table, instant); + return; + } + HoodieCommitMetadata metadata = CompactHelpers.getInstance().createCompactionMetadata( table, instant, HoodieListData.eager(statuses), writeClient.getConfig().getSchema()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index b97306ff29874..10b1499b5465c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -70,12 +70,10 @@ public HoodieFlinkCompactor(AsyncCompactionService service) { } public static void main(String[] args) throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - FlinkCompactionConfig cfg = getFlinkCompactionConfig(args); Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg); - AsyncCompactionService service = new AsyncCompactionService(cfg, conf, env); + AsyncCompactionService service = new AsyncCompactionService(cfg, conf); new HoodieFlinkCompactor(service).start(cfg.serviceMode); } @@ -158,20 +156,14 @@ public static class AsyncCompactionService extends HoodieAsyncTableService { */ private final HoodieFlinkTable table; - /** - * Flink Execution Environment. - */ - private final StreamExecutionEnvironment env; - /** * Executor Service. */ private final ExecutorService executor; - public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception { + public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf) throws Exception { this.cfg = cfg; this.conf = conf; - this.env = env; this.executor = Executors.newFixedThreadPool(1); // create metaClient @@ -305,6 +297,7 @@ private void compact() throws Exception { } table.getMetaClient().reloadActiveTimeline(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.addSource(new CompactionPlanSourceFunction(compactionPlans)) .name("compaction_source") .uid("uid_compaction_source") diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java index c20b263fa36da..155f0e6905e1f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java @@ -90,13 +90,14 @@ public void close() { // ------------------------------------------------------------------------- /** - * Initialize the message bus, would clean all the messages + * Initialize the message bus, would keep all the messages. * *

This expects to be called by the driver. */ public void bootstrap() throws IOException { - fs.delete(path, true); - fs.mkdirs(path); + if (!fs.exists(path)) { + fs.mkdirs(path); + } } public void startInstant(String instant) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java index 5fa3d1ab9a0a2..4e0c08b104657 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java @@ -42,7 +42,8 @@ public BucketIndexPartitioner(int bucketNum, String indexKeyFields) { @Override public int partition(HoodieKey key, int numPartitions) { int curBucket = BucketIdentifier.getBucketId(key, indexKeyFields, bucketNum); - int globalHash = (key.getPartitionPath() + curBucket).hashCode() & Integer.MAX_VALUE; - return BucketIdentifier.mod(globalHash, numPartitions); + int partitionIndex = (key.getPartitionPath().hashCode() & Integer.MAX_VALUE) % numPartitions; + int globalIndex = partitionIndex + curBucket; + return BucketIdentifier.mod(globalIndex, numPartitions); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java index 2dd86d652869f..8fadc682dcac6 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java @@ -18,7 +18,6 @@ package org.apache.hudi.source; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -37,6 +36,7 @@ import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.StreamerUtil; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.table.types.logical.RowType; @@ -398,6 +398,17 @@ private List getArchivedMetadata( return Collections.emptyList(); } + private HoodieTimeline getReadTimeline(HoodieTableMetaClient metaClient) { + HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(); + return filterInstantsByCondition(timeline); + } + + private HoodieTimeline getArchivedReadTimeline(HoodieTableMetaClient metaClient, String startInstant) { + HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(startInstant, false); + HoodieTimeline archivedCompleteTimeline = archivedTimeline.getCommitsTimeline().filterCompletedInstants(); + return filterInstantsByCondition(archivedCompleteTimeline); + } + /** * Returns the instants with a given issuedInstant to start from. * @@ -446,6 +457,23 @@ private Stream maySkipCompaction(Stream instants) : instants; } + /** + * Filters out the unnecessary instants by user specified condition. + * + * @param timeline The timeline + * + * @return the filtered timeline + */ + @VisibleForTesting + public HoodieTimeline filterInstantsByCondition(HoodieTimeline timeline) { + final HoodieTimeline oriTimeline = timeline; + if (this.skipCompaction) { + // the compaction commit uses 'commit' as action which is tricky + timeline = timeline.filter(instant -> !instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)); + } + return timeline; + } + private static List mergeList(List list1, List list2) { List merged = new ArrayList<>(list1); merged.addAll(list2); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java index 4383b42e9f8d9..fac507cb7db6f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HiveSchemaUtils.java @@ -181,9 +181,11 @@ private static DataType toFlinkPrimitiveType(PrimitiveTypeInfo hiveType) { */ public static List toHiveFieldSchema(TableSchema schema, boolean withOperationField) { List columns = new ArrayList<>(); - Collection metaFields = withOperationField - ? HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION // caution that the set may break sequence - : HoodieRecord.HOODIE_META_COLUMNS; + Collection metaFields = new ArrayList<>(HoodieRecord.HOODIE_META_COLUMNS); + if (withOperationField) { + metaFields.add(HoodieRecord.OPERATION_METADATA_FIELD); + } + for (String metaField : metaFields) { columns.add(new FieldSchema(metaField, "string", null)); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java index 956d61cc3c2a4..cf7ff64e5c620 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java @@ -19,11 +19,17 @@ package org.apache.hudi.table.catalog; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; +import org.apache.hudi.configuration.OptionsResolver; +import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.DataTypeUtils; import org.apache.hudi.util.StreamerUtil; import org.apache.avro.Schema; @@ -56,8 +62,8 @@ import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.types.DataType; import org.apache.flink.util.CollectionUtil; -import org.apache.flink.util.StringUtils; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -146,7 +152,7 @@ public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistE @Override public boolean databaseExists(String databaseName) throws CatalogException { - checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName)); + checkArgument(!StringUtils.isNullOrEmpty(databaseName)); return listDatabases().contains(databaseName); } @@ -238,11 +244,17 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep Map options = TableOptionProperties.loadFromProperties(path, hadoopConf); final Schema latestSchema = getLatestTableSchema(path); if (latestSchema != null) { + List pkColumns = TableOptionProperties.getPkColumns(options); + // if the table is initialized from spark, the write schema is nullable for pk columns. + DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable( + AvroSchemaConverter.convertToDataType(latestSchema), pkColumns); org.apache.flink.table.api.Schema.Builder builder = org.apache.flink.table.api.Schema.newBuilder() - .fromRowDataType(AvroSchemaConverter.convertToDataType(latestSchema)); + .fromRowDataType(tableDataType); final String pkConstraintName = TableOptionProperties.getPkConstraintName(options); - if (pkConstraintName != null) { - builder.primaryKeyNamed(pkConstraintName, TableOptionProperties.getPkColumns(options)); + if (!StringUtils.isNullOrEmpty(pkConstraintName)) { + builder.primaryKeyNamed(pkConstraintName, pkColumns); + } else if (!CollectionUtils.isNullOrEmpty(pkColumns)) { + builder.primaryKey(pkColumns); } final org.apache.flink.table.api.Schema schema = builder.build(); return CatalogTable.of( @@ -297,6 +309,21 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boo options.put(TableOptionProperties.PK_CONSTRAINT_NAME, resolvedSchema.getPrimaryKey().get().getName()); options.put(TableOptionProperties.PK_COLUMNS, pkColumns); + // check preCombine + final String preCombineField = conf.getString(FlinkOptions.PRECOMBINE_FIELD); + if (!resolvedSchema.getColumnNames().contains(preCombineField)) { + if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) { + throw new HoodieValidationException("Option '" + FlinkOptions.PRECOMBINE_FIELD.key() + + "' is required for payload class: " + DefaultHoodieRecordPayload.class.getName()); + } + if (preCombineField.equals(FlinkOptions.PRECOMBINE_FIELD.defaultValue())) { + conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.NO_PRE_COMBINE); + } else if (!preCombineField.equals(FlinkOptions.NO_PRE_COMBINE)) { + throw new HoodieValidationException("Field " + preCombineField + " does not exist in the table schema." + + "Please check '" + FlinkOptions.PRECOMBINE_FIELD.key() + "' option."); + } + } + if (resolvedTable.isPartitioned()) { final String partitions = String.join(",", resolvedTable.getPartitionKeys()); conf.setString(FlinkOptions.PARTITION_PATH_FIELD, partitions); @@ -306,7 +333,7 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boo try { StreamerUtil.initTableIfNotExists(conf); // prepare the non-table-options properties - if (!StringUtils.isNullOrWhitespaceOnly(resolvedTable.getComment())) { + if (!StringUtils.isNullOrEmpty(resolvedTable.getComment())) { options.put(TableOptionProperties.COMMENT, resolvedTable.getComment()); } TableOptionProperties.createProperties(tablePathStr, hadoopConf, options); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java index 8a34695ff3b29..4cd873223d43b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -377,12 +378,22 @@ private Table translateSparkTable2Flink(ObjectPath tablePath, Table hiveTable) { String path = hiveTable.getSd().getLocation(); parameters.put(PATH.key(), path); if (!parameters.containsKey(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { - Path hoodieTablePath = new Path(path); - boolean hiveStyle = Arrays.stream(FSUtils.getFs(hoodieTablePath, hiveConf).listStatus(hoodieTablePath)) - .map(fileStatus -> fileStatus.getPath().getName()) - .filter(f -> !f.equals(".hoodie") && !f.equals("default")) - .anyMatch(FilePathUtils::isHiveStylePartitioning); - parameters.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), String.valueOf(hiveStyle)); + // read the table config first + final boolean hiveStyle; + HoodieTableConfig tableConfig = StreamerUtil.getTableConfig(path, hiveConf); + if (tableConfig != null && tableConfig.contains(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { + hiveStyle = Boolean.parseBoolean(tableConfig.getHiveStylePartitioningEnable()); + } else { + // fallback to the partition path pattern + Path hoodieTablePath = new Path(path); + hiveStyle = Arrays.stream(FSUtils.getFs(hoodieTablePath, hiveConf).listStatus(hoodieTablePath)) + .map(fileStatus -> fileStatus.getPath().getName()) + .filter(f -> !f.equals(".hoodie") && !f.equals("default")) + .anyMatch(FilePathUtils::isHiveStylePartitioning); + } + if (hiveStyle) { + parameters.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), "true"); + } } client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); } catch (Exception e) { @@ -399,11 +410,11 @@ public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistExcep String path = hiveTable.getSd().getLocation(); Map parameters = hiveTable.getParameters(); Schema latestTableSchema = StreamerUtil.getLatestTableSchema(path, hiveConf); - String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key()); - List pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr) - ? null : StringUtils.split(pkColumnsStr, ","); org.apache.flink.table.api.Schema schema; if (latestTableSchema != null) { + String pkColumnsStr = parameters.get(FlinkOptions.RECORD_KEY_FIELD.key()); + List pkColumns = StringUtils.isNullOrEmpty(pkColumnsStr) + ? null : StringUtils.split(pkColumnsStr, ","); // if the table is initialized from spark, the write schema is nullable for pk columns. DataType tableDataType = DataTypeUtils.ensureColumnsAsNonNullable( AvroSchemaConverter.convertToDataType(latestTableSchema), pkColumns); @@ -578,7 +589,7 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table, serdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(!useRealTimeInputFormat)); serdeProperties.put("serialization.format", "1"); - serdeProperties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark(catalogTable, hiveConf, properties, partitionKeys)); + serdeProperties.putAll(TableOptionProperties.translateFlinkTableProperties2Spark(catalogTable, hiveConf, properties, partitionKeys, withOperationField)); sd.setSerdeInfo(new SerDeInfo(null, serDeClassName, serdeProperties)); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java index a0864bbf3773b..6e327bdc61202 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.catalog; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.configuration.FlinkOptions; @@ -28,6 +29,8 @@ import org.apache.avro.Schema; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -39,7 +42,9 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Date; import java.util.HashMap; @@ -49,6 +54,7 @@ import java.util.stream.Collectors; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD; import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME; /** @@ -168,8 +174,10 @@ public static Map translateFlinkTableProperties2Spark( CatalogTable catalogTable, Configuration hadoopConf, Map properties, - List partitionKeys) { - Schema schema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPhysicalRowDataType().getLogicalType()); + List partitionKeys, + boolean withOperationField) { + RowType rowType = supplementMetaFields((RowType) catalogTable.getSchema().toPhysicalRowDataType().getLogicalType(), withOperationField); + Schema schema = AvroSchemaConverter.convertToSchema(rowType); MessageType messageType = TableSchemaResolver.convertAvroSchemaToParquet(schema, hadoopConf); String sparkVersion = catalogTable.getOptions().getOrDefault(SPARK_VERSION, DEFAULT_SPARK_VERSION); Map sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties( @@ -184,6 +192,19 @@ public static Map translateFlinkTableProperties2Spark( e -> e.getKey().equalsIgnoreCase(FlinkOptions.TABLE_TYPE.key()) ? VALUE_MAPPING.get(e.getValue()) : e.getValue())); } + private static RowType supplementMetaFields(RowType rowType, boolean withOperationField) { + Collection metaFields = new ArrayList<>(HoodieRecord.HOODIE_META_COLUMNS); + if (withOperationField) { + metaFields.add(OPERATION_METADATA_FIELD); + } + ArrayList rowFields = new ArrayList<>(); + for (String metaField : metaFields) { + rowFields.add(new RowType.RowField(metaField, new VarCharType(10000))); + } + rowFields.addAll(rowType.getFields()); + return new RowType(false, rowFields); + } + public static Map translateSparkTableProperties2Flink(Map options) { if (options.containsKey(CONNECTOR.key())) { return options; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index c9b6561bdef20..f54cd86ef3b57 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -169,10 +169,10 @@ public void open(MergeOnReadInputSplit split) throws IOException { this.closed = false; this.hadoopConf = HadoopConfigurations.getHadoopConf(this.conf); if (!(split.getLogPaths().isPresent() && split.getLogPaths().get().size() > 0)) { - if (split.getInstantRange() != null) { + if (split.getInstantRange().isPresent()) { // base file only with commit time filtering this.iterator = new BaseFileOnlyFilteringIterator( - split.getInstantRange(), + split.getInstantRange().get(), this.tableState.getRequiredRowType(), getReader(split.getBasePath().get(), getRequiredPosWithCommitTime(this.requiredPos))); } else { @@ -506,11 +506,11 @@ static class BaseFileOnlyFilteringIterator implements RecordIterator { private RowData currentRecord; BaseFileOnlyFilteringIterator( - Option instantRange, + InstantRange instantRange, RowType requiredRowType, ParquetColumnarRowSplitReader reader) { this.reader = reader; - this.instantRange = instantRange.orElse(null); + this.instantRange = instantRange; int[] positions = IntStream.range(1, 1 + requiredRowType.getFieldCount()).toArray(); projection = RowDataProjection.instance(requiredRowType, positions); } @@ -519,12 +519,8 @@ static class BaseFileOnlyFilteringIterator implements RecordIterator { public boolean reachedEnd() throws IOException { while (!this.reader.reachedEnd()) { currentRecord = this.reader.nextRecord(); - if (instantRange != null) { - boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString()); - if (isInRange) { - return false; - } - } else { + boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString()); + if (isInRange) { return false; } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/JsonDeserializationFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/JsonDeserializationFunction.java new file mode 100644 index 0000000000000..ae5a45d7c2149 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/JsonDeserializationFunction.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.util; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.common.TimestampFormat; +import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +import java.nio.charset.StandardCharsets; + +/** + * Wrapper function that manages the lifecycle of the JSON deserialization schema. + */ +public final class JsonDeserializationFunction + extends AbstractRichFunction + implements MapFunction { + private final JsonRowDataDeserializationSchema deserializationSchema; + + public static JsonDeserializationFunction getInstance(Configuration conf) { + // Read from file source + RowType rowType = + (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) + .getLogicalType(); + return getInstance(rowType); + } + + public static JsonDeserializationFunction getInstance(RowType rowType) { + JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( + rowType, + InternalTypeInfo.of(rowType), + false, + true, + TimestampFormat.ISO_8601 + ); + return new JsonDeserializationFunction(deserializationSchema); + } + + public JsonDeserializationFunction(JsonRowDataDeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + this.deserializationSchema.open(null); + } + + @Override + public RowData map(String record) throws Exception { + return deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)); + } +} diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java index ecebd1adcdbc8..ff2903c0a7339 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/RowDataToAvroConverters.java @@ -37,7 +37,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -176,7 +175,8 @@ public Object convert(Schema schema, Object object) { @Override public Object convert(Schema schema, Object object) { - return ChronoUnit.MICROS.between(Instant.EPOCH, ((TimestampData) object).toInstant()); + Instant instant = ((TimestampData) object).toInstant(); + return Math.addExact(Math.multiplyExact(instant.getEpochSecond(), 1000_000), instant.getNano() / 1000); } }; } else { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 98664c6dc3bd2..d377811676733 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -56,6 +57,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.BufferedReader; import java.io.IOException; import java.io.StringReader; @@ -188,6 +191,7 @@ public static HoodieTableMetaClient initTableIfNotExists( .setTableCreateSchema(conf.getString(FlinkOptions.SOURCE_AVRO_SCHEMA)) .setTableType(conf.getString(FlinkOptions.TABLE_TYPE)) .setTableName(conf.getString(FlinkOptions.TABLE_NAME)) + .setDatabaseName(conf.getString(FlinkOptions.DATABASE_NAME)) .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null)) .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME)) .setPreCombineField(OptionsResolver.getPreCombineField(conf)) @@ -263,6 +267,23 @@ public static HoodieTableMetaClient createMetaClient(Configuration conf) { return createMetaClient(conf.getString(FlinkOptions.PATH), HadoopConfigurations.getHadoopConf(conf)); } + /** + * Returns the table config or null if the table does not exist. + */ + @Nullable + public static HoodieTableConfig getTableConfig(String basePath, org.apache.hadoop.conf.Configuration hadoopConf) { + FileSystem fs = FSUtils.getFs(basePath, hadoopConf); + Path metaPath = new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + try { + if (fs.exists(metaPath)) { + return new HoodieTableConfig(fs, metaPath.toString(), null); + } + } catch (IOException e) { + throw new HoodieIOException("Get table config error", e); + } + return null; + } + /** * Returns the median instant time between the given two instant time. */ diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java index 6ab4b1b6e0d48..8047122374d6d 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java @@ -28,6 +28,7 @@ import org.apache.hudi.sink.utils.Pipelines; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.HoodiePipeline; +import org.apache.hudi.util.JsonDeserializationFunction; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.FlinkMiniCluster; import org.apache.hudi.utils.TestConfigurations; @@ -43,15 +44,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.fs.Path; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.FileProcessingMode; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.TestLogger; import org.junit.jupiter.api.Test; @@ -61,7 +59,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.io.File; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -216,13 +213,6 @@ private void testWriteToHoodie( (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) .getLogicalType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601 - ); String sourcePath = Objects.requireNonNull(Thread.currentThread() .getContextClassLoader().getResource("test_source.data")).toString(); @@ -238,7 +228,7 @@ private void testWriteToHoodie( dataStream = execEnv // use PROCESS_CONTINUOUSLY mode to trigger checkpoint .readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .map(JsonDeserializationFunction.getInstance(rowType)) .setParallelism(1); } else { dataStream = execEnv @@ -246,7 +236,7 @@ private void testWriteToHoodie( .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), checkpoints)) .name("continuous_file_source") .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .map(JsonDeserializationFunction.getInstance(rowType)) .setParallelism(4); } @@ -285,13 +275,6 @@ private void testWriteToHoodieWithCluster( (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) .getLogicalType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601 - ); String sourcePath = Objects.requireNonNull(Thread.currentThread() .getContextClassLoader().getResource("test_source.data")).toString(); @@ -300,7 +283,7 @@ private void testWriteToHoodieWithCluster( .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), checkpoints)) .name("continuous_file_source") .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .map(JsonDeserializationFunction.getInstance(rowType)) .setParallelism(4); OptionsInference.setupSinkTasks(conf, execEnv.getParallelism()); @@ -384,18 +367,7 @@ public void testHoodiePipelineBuilderSink() throws Exception { options.put(FlinkOptions.PATH.key(), tempFile.toURI().toString()); options.put(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH.key(), Objects.requireNonNull(Thread.currentThread().getContextClassLoader().getResource("test_read_schema.avsc")).toString()); Configuration conf = Configuration.fromMap(options); - // Read from file source - RowType rowType = - (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)) - .getLogicalType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601 - ); String sourcePath = Objects.requireNonNull(Thread.currentThread() .getContextClassLoader().getResource("test_source.data")).toString(); @@ -408,7 +380,7 @@ public void testHoodiePipelineBuilderSink() throws Exception { .addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), 2)) .name("continuous_file_source") .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8))) + .map(JsonDeserializationFunction.getInstance(conf)) .setParallelism(4); //sink to hoodie table use low-level sink api. diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index 07f560c14f1be..0dc50fd9bee42 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -165,6 +165,22 @@ public void testCheckpointCompleteWithPartialEvents() { assertThat("Commits the instant with partial events anyway", lastCompleted, is(instant)); } + @Test + public void testRecommitWithPartialUncommittedEvents() { + final CompletableFuture future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(1, future); + String instant = coordinator.getInstant(); + String lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath()); + assertNull(lastCompleted, "Returns early for empty write results"); + WriteMetadataEvent event1 = createOperatorEvent(0, instant, "par1", false, 0.2); + event1.setBootstrap(true); + WriteMetadataEvent event2 = WriteMetadataEvent.emptyBootstrap(1); + coordinator.handleEventFromOperator(0, event1); + coordinator.handleEventFromOperator(1, event2); + lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath()); + assertThat("Recommits the instant with partial uncommitted events", lastCompleted, is(instant)); + } + @Test public void testHiveSyncInvoked() throws Exception { // reset diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java new file mode 100644 index 0000000000000..6fc0c840be8fb --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.sink.bucket; + + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.FileCreateUtils; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.sink.clustering.FlinkClusteringConfig; +import org.apache.hudi.util.CompactionUtil; +import org.apache.hudi.util.StreamerUtil; +import org.apache.hudi.utils.FlinkMiniCluster; +import org.apache.hudi.utils.TestConfigurations; +import org.apache.hudi.utils.TestData; +import org.apache.hudi.utils.TestSQL; + +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * Integration test cases for {@link BucketStreamWriteFunction}. + */ +@ExtendWith(FlinkMiniCluster.class) +public class ITTestBucketStreamWrite { + + private static final Map EXPECTED = new HashMap<>(); + + static { + EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1000,par1, id2,par1,id2,Stephen,33,2000,par1]"); + EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3000,par2, id4,par2,id4,Fabian,31,4000,par2]"); + EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5000,par3, id6,par3,id6,Emma,20,6000,par3]"); + EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7000,par4, id8,par4,id8,Han,56,8000,par4]"); + } + + @TempDir + File tempFile; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testBucketStreamWriteAfterRollbackFirstFileGroupCreation(boolean isCow) throws Exception { + // this test is to ensure that the correct fileId can be fetched when recovering from a rollover when a new + // fileGroup is created for a bucketId + String tablePath = tempFile.getAbsolutePath(); + doWrite(tablePath, isCow); + doDeleteCommit(tablePath, isCow); + doWrite(tablePath, isCow); + doWrite(tablePath, isCow); + + if (isCow) { + TestData.checkWrittenData(tempFile, EXPECTED, 4); + } else { + FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), new org.apache.hadoop.conf.Configuration()); + TestData.checkWrittenDataMOR(fs, tempFile, EXPECTED, 4); + } + } + + private static void doDeleteCommit(String tablePath, boolean isCow) throws Exception { + // make configuration and setAvroSchema + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + cfg.path = tablePath; + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableType().name()); + + // set the table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + // should only contain one instant + HoodieTimeline activeCompletedTimeline = metaClient.getActiveTimeline().filterCompletedInstants(); + assertEquals(1, activeCompletedTimeline.getInstants().count()); + + // rollback path structure: tablePath/.hoodie/.temp/${commitInstant}/${partition}/${fileGroup}_${fileInstant}.parquet.marker.APPEND + HoodieInstant instant = activeCompletedTimeline.getInstants().collect(Collectors.toList()).get(0); + String commitInstant = instant.getTimestamp(); + String filename = activeCompletedTimeline.getInstants().collect(Collectors.toList()).get(0).getFileName(); + + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata + .fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + + // delete successful commit to simulate an unsuccessful write + FileSystem fs = metaClient.getFs(); + Path path = new Path(metaClient.getMetaPath() + Path.SEPARATOR + filename); + fs.delete(path); + + // marker types are different for COW and MOR + IOType ioType = isCow ? IOType.CREATE : IOType.APPEND; + + commitMetadata.getFileIdAndRelativePaths().forEach((fileId, relativePath) -> { + // hacky way to reconstruct markers ¯\_(ツ)_/¯ + String[] partitionFileNameSplit = relativePath.split("/"); + String fileInstant = FSUtils.getCommitTime(partitionFileNameSplit[1]); + String partition = partitionFileNameSplit[0]; + String writeToken = isCow ? getWriteToken(partitionFileNameSplit[1]) : FSUtils.getWriteTokenFromLogPath(new Path(relativePath)); + try { + FileCreateUtils.createMarkerFile(tablePath, partition, commitInstant, fileInstant, fileId, ioType, writeToken); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private static String getWriteToken(String relativeFilePath) { + Pattern writeTokenPattern = Pattern.compile("_((\\d+)-(\\d+)-(\\d+))_"); + Matcher matcher = writeTokenPattern.matcher(relativeFilePath); + if (!matcher.find()) { + throw new RuntimeException("Invalid relative file path: " + relativeFilePath); + } + return matcher.group(1); + } + + private static void doWrite(String path, boolean isCow) throws InterruptedException, ExecutionException { + // create hoodie table and perform writes + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + TableEnvironment tableEnv = TableEnvironmentImpl.create(settings); + Map options = new HashMap<>(); + options.put(FlinkOptions.PATH.key(), path); + + // use bucket index + options.put(FlinkOptions.TABLE_TYPE.key(), isCow ? FlinkOptions.TABLE_TYPE_COPY_ON_WRITE : FlinkOptions.TABLE_TYPE_MERGE_ON_READ); + options.put(FlinkOptions.INDEX_TYPE.key(), IndexType.BUCKET.name()); + options.put(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.key(), "1"); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); + tableEnv.executeSql(hoodieTableDDL); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(3); + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java index a66874c486414..cdfb48fc81307 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bulk/TestRowDataKeyGen.java @@ -20,6 +20,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieKeyException; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.table.HoodieTableFactory; import org.apache.hudi.utils.TestConfigurations; @@ -32,6 +33,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.sql.Timestamp; + import static org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; import static org.apache.hudi.utils.TestData.insertRow; import static org.hamcrest.CoreMatchers.is; @@ -165,4 +168,23 @@ void testDateBasedKeyGenerator(String partitionFormat) { assertThat(keyGen2.getPartitionPath(rowData2), is("dt=" + expectedPartition2)); assertThat(keyGen2.getPartitionPath(rowData3), is("dt=" + expectedPartition3)); } + + @Test + void testRecoredKeyContainsTimestamp() { + Configuration conf = TestConfigurations.getDefaultConf("path1"); + conf.setString(FlinkOptions.RECORD_KEY_FIELD, "uuid,ts"); + conf.setString(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "true"); + Timestamp ts = new Timestamp(1675841687000L); + final RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23, + TimestampData.fromTimestamp(ts), StringData.fromString("par1")); + final RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE); + + assertThat(keyGen1.getRecordKey(rowData1), is("uuid:id1,ts:" + ts.toLocalDateTime().toString())); + + conf.setString(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), "false"); + final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE); + + assertThat(keyGen2.getRecordKey(rowData1), is("uuid:id1,ts:1675841687000")); + + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java index f2273e40a26db..7dcd0cec1c3dd 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -50,6 +51,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -208,14 +210,13 @@ public void testHoodieFlinkClusteringService() throws Exception { TimeUnit.SECONDS.sleep(3); // Make configuration and setAvroSchema. - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); FlinkClusteringConfig cfg = new FlinkClusteringConfig(); cfg.path = tempFile.getAbsolutePath(); cfg.minClusteringIntervalSeconds = 3; cfg.schedule = true; Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); - HoodieFlinkClusteringJob.AsyncClusteringService asyncClusteringService = new HoodieFlinkClusteringJob.AsyncClusteringService(cfg, conf, env); + HoodieFlinkClusteringJob.AsyncClusteringService asyncClusteringService = new HoodieFlinkClusteringJob.AsyncClusteringService(cfg, conf); asyncClusteringService.start(null); // wait for the asynchronous commit to finish @@ -291,4 +292,133 @@ public void testHoodieFlinkClusteringSchedule() throws Exception { assertTrue(scheduled, "2 delta commits, the clustering plan should be scheduled"); } + + @Test + public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception { + // Create hoodie table and insert into data. + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + TableEnvironment tableEnv = TableEnvironmentImpl.create(settings); + tableEnv.getConfig().getConfiguration() + .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); + Map options = new HashMap<>(); + options.put(FlinkOptions.HIVE_STYLE_PARTITIONING.key(), "false"); + options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + + // use append mode + options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value()); + options.put(FlinkOptions.INSERT_CLUSTER.key(), "false"); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options); + tableEnv.executeSql(hoodieTableDDL); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(3); + + // Make configuration and setAvroSchema. + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + FlinkClusteringConfig cfg = new FlinkClusteringConfig(); + cfg.path = tempFile.getAbsolutePath(); + cfg.targetPartitions = 4; + Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + + // create metaClient + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + + // set the table name + conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName()); + + // set record key field + conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp()); + // set partition field + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp()); + + long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); + conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); + conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition"); + + // set archive commits + conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS.key(), 2); + conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS.key(), 1); + conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), 0); + + // set table schema + CompactionUtil.setAvroSchema(conf, metaClient); + + // judge whether have operation + // To compute the clustering instant time and do clustering. + String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime(); + + HoodieFlinkWriteClient writeClient = FlinkWriteClients.createWriteClient(conf); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty()); + + assertTrue(scheduled, "The clustering plan should be scheduled"); + + // fetch the instant based on the configured execution sequence + table.getMetaClient().reloadActiveTimeline(); + HoodieTimeline timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(i -> i.getState() == HoodieInstant.State.REQUESTED); + + // generate clustering plan + // should support configurable commit metadata + Option> clusteringPlanOption = ClusteringUtils.getClusteringPlan( + table.getMetaClient(), timeline.lastInstant().get()); + + HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); + + // Mark instant as clustering inflight + HoodieInstant instant = HoodieTimeline.getReplaceCommitRequestedInstant(clusteringInstantTime); + table.getActiveTimeline().transitionReplaceRequestedToInflight(instant, Option.empty()); + + final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); + final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema); + final RowType rowType = (RowType) rowDataType.getLogicalType(); + + DataStream dataStream = + env.addSource(new ClusteringPlanSourceFunction(clusteringInstantTime, clusteringPlan)) + .name("clustering_source") + .uid("uid_clustering_source") + .rebalance() + .transform( + "clustering_task", + TypeInformation.of(ClusteringCommitEvent.class), + new ClusteringOperator(conf, rowType)) + .setParallelism(clusteringPlan.getInputGroups().size()); + + ExecNodeUtil.setManagedMemoryWeight( + dataStream.getTransformation(), + conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L); + + // keep pending clustering, not committing clustering + dataStream + .addSink(new DiscardingSink<>()) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + // wait for the asynchronous commit to finish + TimeUnit.SECONDS.sleep(3); + + // archive the first commit, retain the second commit before the inflight replacecommit + writeClient.archive(); + + scheduled = writeClient.scheduleClusteringAtInstant(HoodieActiveTimeline.createNewInstantTime(), Option.empty()); + + assertTrue(scheduled, "The clustering plan should be scheduled"); + table.getMetaClient().reloadActiveTimeline(); + timeline = table.getActiveTimeline().filterPendingReplaceTimeline() + .filter(i -> i.getState() == HoodieInstant.State.REQUESTED); + + // clustering plan has no previous file slice generated by previous pending clustering + assertFalse(ClusteringUtils.getClusteringPlan(table.getMetaClient(), timeline.lastInstant().get()).get() + .getRight().getInputGroups() + .stream().anyMatch(g -> g.getSlices() + .stream().anyMatch(f -> clusteringInstantTime.equals(FSUtils.getCommitTime(f.getDataFilePath()))))); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index 6157b5e901130..0ad78890aad37 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -191,7 +191,6 @@ public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exce tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await(); // Make configuration and setAvroSchema. - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); FlinkCompactionConfig cfg = new FlinkCompactionConfig(); cfg.path = tempFile.getAbsolutePath(); cfg.minCompactionIntervalSeconds = 3; @@ -200,7 +199,7 @@ public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exce conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ"); conf.setInteger(FlinkOptions.COMPACTION_TASKS.key(), FlinkMiniCluster.DEFAULT_PARALLELISM); - HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new HoodieFlinkCompactor.AsyncCompactionService(cfg, conf, env); + HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new HoodieFlinkCompactor.AsyncCompactionService(cfg, conf); asyncCompactionService.start(null); // wait for the asynchronous commit to finish diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java index b42fd2c04a3c0..1b5481e86bcea 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java @@ -18,10 +18,14 @@ package org.apache.hudi.source; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.utils.TestConfigurations; @@ -31,9 +35,13 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertIterableEquals; @@ -85,4 +93,38 @@ void testFilterInstantsWithRange() { assertIterableEquals(Arrays.asList(commit1, commit2, commit3), instantRange3); } + @Test + void testFilterInstantsByCondition() throws IOException { + HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient, true); + Configuration conf = TestConfigurations.getDefaultConf(basePath); + IncrementalInputSplits iis = IncrementalInputSplits.builder() + .conf(conf) + .path(new Path(basePath)) + .rowType(TestConfigurations.ROW_TYPE) + .build(); + + HoodieInstant commit1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); + HoodieInstant commit2 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); + HoodieInstant commit3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "3"); + timeline.createNewInstant(commit1); + timeline.createNewInstant(commit2); + timeline.createNewInstant(commit3); + commit3 = timeline.transitionReplaceRequestedToInflight(commit3, Option.empty()); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata( + new ArrayList<>(), + new HashMap<>(), + Option.empty(), + WriteOperationType.CLUSTER, + "", + HoodieTimeline.REPLACE_COMMIT_ACTION); + timeline.transitionReplaceInflightToComplete( + HoodieTimeline.getReplaceCommitInflightInstant(commit3.getTimestamp()), + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + timeline = timeline.reload(); + + conf.set(FlinkOptions.READ_END_COMMIT, "3"); + HoodieTimeline resTimeline = iis.filterInstantsByCondition(timeline); + // will not filter cluster commit by default + assertEquals(3, resTimeline.getInstants().collect(Collectors.toList()).size()); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index afaeee34406db..aa866b479571b 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -58,6 +58,7 @@ import java.io.File; import java.time.ZoneId; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -70,7 +71,11 @@ import static org.apache.hudi.utils.TestConfigurations.catalog; import static org.apache.hudi.utils.TestConfigurations.sql; +import static org.apache.hudi.utils.TestData.array; +import static org.apache.hudi.utils.TestData.assertRowsEqualsUnordered; import static org.apache.hudi.utils.TestData.assertRowsEquals; +import static org.apache.hudi.utils.TestData.map; +import static org.apache.hudi.utils.TestData.row; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -608,14 +613,16 @@ void testWriteAndReadWithTimestampMicros(ExecMode execMode) throws Exception { + "(1,'Danny',TIMESTAMP '2021-12-01 01:02:01.100001'),\n" + "(2,'Stephen',TIMESTAMP '2021-12-02 03:04:02.200002'),\n" + "(3,'Julian',TIMESTAMP '2021-12-03 13:14:03.300003'),\n" - + "(4,'Fabian',TIMESTAMP '2021-12-04 15:16:04.400004')"; + + "(4,'Fabian',TIMESTAMP '2021-12-04 15:16:04.400004'),\n" + + "(5,'Tom',TIMESTAMP '2721-12-04 15:16:04.500005')"; execInsertSql(streamTableEnv, insertInto); final String expected = "[" + "+I[1, Danny, 2021-12-01T01:02:01.100001], " + "+I[2, Stephen, 2021-12-02T03:04:02.200002], " + "+I[3, Julian, 2021-12-03T13:14:03.300003], " - + "+I[4, Fabian, 2021-12-04T15:16:04.400004]]"; + + "+I[4, Fabian, 2021-12-04T15:16:04.400004], " + + "+I[5, Tom, 2721-12-04T15:16:04.500005]]"; List result = execSelectSql(streamTableEnv, "select * from t1", execMode); assertRowsEquals(result, expected); @@ -1265,11 +1272,11 @@ void testParquetComplexTypes(String operation) { List result = CollectionUtil.iterableToList( () -> tableEnv.sqlQuery("select * from t1").execute().collect()); - final String expected = "[" - + "+I[1, [abc1, def1], {abc1=1, def1=3}, +I[1, abc1]], " - + "+I[2, [abc2, def2], {def2=3, abc2=1}, +I[2, abc2]], " - + "+I[3, [abc3, def3], {def3=3, abc3=1}, +I[3, abc3]]]"; - assertRowsEquals(result, expected); + List expected = Arrays.asList( + row(1, array("abc1", "def1"), map("abc1", 1, "def1", 3), row(1, "abc1")), + row(2, array("abc2", "def2"), map("abc2", 1, "def2", 3), row(2, "abc2")), + row(3, array("abc3", "def3"), map("abc3", 1, "def3", 3), row(3, "abc3"))); + assertRowsEqualsUnordered(result, expected); } @ParameterizedTest @@ -1294,11 +1301,11 @@ void testParquetComplexNestedRowTypes(String operation) { List result = CollectionUtil.iterableToList( () -> tableEnv.sqlQuery("select * from t1").execute().collect()); - final String expected = "[" - + "+I[1, [abc1, def1], [1, 1], {abc1=1, def1=3}, +I[[abc1, def1], +I[1, abc1]]], " - + "+I[2, [abc2, def2], [2, 2], {def2=3, abc2=1}, +I[[abc2, def2], +I[2, abc2]]], " - + "+I[3, [abc3, def3], [3, 3], {def3=3, abc3=1}, +I[[abc3, def3], +I[3, abc3]]]]"; - assertRowsEquals(result, expected); + List expected = Arrays.asList( + row(1, array("abc1", "def1"), array(1, 1), map("abc1", 1, "def1", 3), row(array("abc1", "def1"), row(1, "abc1"))), + row(2, array("abc2", "def2"), array(2, 2), map("abc2", 1, "def2", 3), row(array("abc2", "def2"), row(2, "abc2"))), + row(3, array("abc3", "def3"), array(3, 3), map("abc3", 1, "def3", 3), row(array("abc3", "def3"), row(3, "abc3")))); + assertRowsEqualsUnordered(result, expected); } @ParameterizedTest diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java index 8e23ef9d63bcb..98348d7abc797 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java @@ -19,6 +19,8 @@ package org.apache.hudi.table.catalog; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.exception.HoodieValidationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; @@ -41,6 +43,7 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -92,7 +95,11 @@ public class TestHoodieCatalog { .getLogicalType() .getTypeRoot() .equals(LogicalTypeRoot.VARCHAR)) { - return Column.physical(col.getName(), DataTypes.STRING()); + DataType dataType = DataTypes.STRING(); + if ("uuid".equals(col.getName())) { + dataType = dataType.notNull(); + } + return Column.physical(col.getName(), dataType); } else { return col; } @@ -132,11 +139,16 @@ void beforeEach() { .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); File testDb = new File(tempFile, TEST_DEFAULT_DATABASE); testDb.mkdir(); + + catalog = new HoodieCatalog("hudi", Configuration.fromMap(getDefaultCatalogOption())); + catalog.open(); + } + + Map getDefaultCatalogOption() { Map catalogOptions = new HashMap<>(); catalogOptions.put(CATALOG_PATH.key(), tempFile.getAbsolutePath()); catalogOptions.put(DEFAULT_DATABASE.key(), TEST_DEFAULT_DATABASE); - catalog = new HoodieCatalog("hudi", Configuration.fromMap(catalogOptions)); - catalog.open(); + return catalogOptions; } @AfterEach @@ -206,6 +218,28 @@ public void testCreateTable() throws Exception { () -> catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, false)); } + @Test + void testCreateTableWithoutPreCombineKey() { + Map options = getDefaultCatalogOption(); + options.put(FlinkOptions.PAYLOAD_CLASS_NAME.key(), DefaultHoodieRecordPayload.class.getName()); + catalog = new HoodieCatalog("hudi", Configuration.fromMap(options)); + catalog.open(); + ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); + assertThrows(HoodieValidationException.class, + () -> catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true), + "Option 'precombine.field' is required for payload class: " + + "org.apache.hudi.common.model.DefaultHoodieRecordPayload"); + + Map options2 = getDefaultCatalogOption(); + options2.put(FlinkOptions.PRECOMBINE_FIELD.key(), "not_exists"); + catalog = new HoodieCatalog("hudi", Configuration.fromMap(options2)); + catalog.open(); + ObjectPath tablePath2 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb2"); + assertThrows(HoodieValidationException.class, + () -> catalog.createTable(tablePath2, EXPECTED_CATALOG_TABLE, true), + "Field not_exists does not exist in the table schema. Please check 'precombine.field' option."); + } + @Test public void testListTable() throws Exception { ObjectPath tablePath1 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1"); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java index ffae71d6b2499..0df1bfa26b919 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieHiveCatalog.java @@ -44,6 +44,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.Collections; @@ -124,6 +125,21 @@ public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Except .collect(Collectors.joining(",")); assertEquals("par1:string", partitionSchema); + // validate spark schema properties + String avroSchemaStr = hiveTable.getParameters().get("spark.sql.sources.schema.part.0"); + String expectedAvroSchemaStr = "" + + "{\"type\":\"struct\",\"fields\":[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"uuid\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"age\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"par1\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}"; + assertEquals(expectedAvroSchemaStr, avroSchemaStr); + // validate catalog table CatalogBaseTable table1 = hoodieCatalog.getTable(tablePath); assertEquals("hudi", table1.getOptions().get(CONNECTOR.key())); @@ -177,6 +193,23 @@ public void testCreateNonHoodieTable() throws TableAlreadyExistException, Databa } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testDropTable(boolean external) throws TableAlreadyExistException, DatabaseNotExistException, TableNotExistException, IOException { + HoodieHiveCatalog catalog = HoodieCatalogTestUtils.createHiveCatalog("myCatalog", external); + catalog.open(); + + CatalogTable catalogTable = new CatalogTableImpl(schema, Collections.singletonMap(FactoryUtil.CONNECTOR.key(), "hudi"), "hudi table"); + catalog.createTable(tablePath, catalogTable, false); + Table table = catalog.getHiveTable(tablePath); + assertEquals(external, Boolean.parseBoolean(table.getParameters().get("EXTERNAL"))); + + catalog.dropTable(tablePath, false); + Path path = new Path(table.getParameters().get(FlinkOptions.PATH.key())); + boolean existing = StreamerUtil.fileExists(FSUtils.getFs(path, new Configuration()), path); + assertEquals(external, existing); + } + @Test public void testAlterTable() throws Exception { Map originOptions = new HashMap<>(); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java index b76905ed8af07..55b505b1587ae 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/format/TestInputFormat.java @@ -26,14 +26,17 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; +import org.apache.hudi.source.IncrementalInputSplits; import org.apache.hudi.table.HoodieTableSource; import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat; import org.apache.hudi.table.format.mor.MergeOnReadInputFormat; +import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.FlinkWriteClients; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestData; +import org.apache.hudi.utils.TestUtils; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.configuration.Configuration; @@ -49,8 +52,10 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -58,6 +63,7 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; /** * Test cases for MergeOnReadInputFormat and ParquetInputFormat. @@ -332,6 +338,50 @@ void testReadWithPartitionPrune(HoodieTableType tableType) throws Exception { assertThat(actual, is(expected)); } + @Test + void testReadBaseFilesWithStartCommit() throws Exception { + beforeEach(HoodieTableType.COPY_ON_WRITE); + + org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf); + + // write base files + TestData.writeData(TestData.DATA_SET_INSERT, conf); + + InputFormat inputFormat = this.tableSource.getInputFormat(true); + assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class)); + + HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); + IncrementalInputSplits incrementalInputSplits = IncrementalInputSplits.builder() + .rowType(TestConfigurations.ROW_TYPE) + .conf(conf) + .path(FilePathUtils.toFlinkPath(metaClient.getBasePathV2())) + .requiredPartitions(new HashSet<>(Arrays.asList("par1", "par2", "par3", "par4"))) + .build(); + + // default read the latest commit + IncrementalInputSplits.Result splits1 = incrementalInputSplits.inputSplits(metaClient, hadoopConf, null); + assertFalse(splits1.isEmpty()); + List result1 = readData(inputFormat, splits1.getInputSplits().toArray(new MergeOnReadInputSplit[0])); + + String actual1 = TestData.rowDataToString(result1); + String expected1 = TestData.rowDataToString(TestData.DATA_SET_INSERT); + assertThat(actual1, is(expected1)); + + // write another commit and read again + TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf); + + // read from the latest commit + String secondCommit = TestUtils.getNthCompleteInstant(metaClient.getBasePath(), 1, false); + conf.setString(FlinkOptions.READ_START_COMMIT, secondCommit); + + IncrementalInputSplits.Result splits2 = incrementalInputSplits.inputSplits(metaClient, hadoopConf, null); + assertFalse(splits2.isEmpty()); + List result2 = readData(inputFormat, splits2.getInputSplits().toArray(new MergeOnReadInputSplit[0])); + String actual2 = TestData.rowDataToString(result2); + String expected2 = TestData.rowDataToString(TestData.DATA_SET_UPDATE_INSERT); + assertThat(actual2, is(expected2)); + } + @Test void testReadChangesMergedMOR() throws Exception { Map options = new HashMap<>(); @@ -632,10 +682,14 @@ private HoodieTableSource getTableSource(Configuration conf) { conf); } - @SuppressWarnings("unchecked, rawtypes") + @SuppressWarnings("rawtypes") private static List readData(InputFormat inputFormat) throws IOException { InputSplit[] inputSplits = inputFormat.createInputSplits(1); + return readData(inputFormat, inputSplits); + } + @SuppressWarnings("unchecked, rawtypes") + private static List readData(InputFormat inputFormat, InputSplit[] inputSplits) throws IOException { List result = new ArrayList<>(); for (InputSplit inputSplit : inputSplits) { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java index 7ee745e0dcc0e..6696295a5d19c 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java @@ -66,8 +66,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -521,6 +523,16 @@ public static void assertRowDataEquals(List rows, List expecte assertThat(rowsString, is(rowDataToString(expected))); } + /** + * Assert that expected and actual collection of rows are equal regardless of the order. + * + * @param expected expected row collection + * @param actual actual row collection + */ + public static void assertRowsEqualsUnordered(Collection expected, Collection actual) { + assertEquals(new HashSet<>(expected), new HashSet<>(actual)); + } + /** * Checks the source data set are written as expected. * @@ -871,4 +883,51 @@ private static BinaryRowData updateAfterRow(Object... fields) { rowData.setRowKind(RowKind.UPDATE_AFTER); return rowData; } + + /** + * Creates row with specified field values. + *

This method is used to define row in convenient way such as: + * + *

 {@code
+   * row(1, array("abc1", "def1"), map("abc1", 1, "def1", 3), row(1, "abc1"))
+   * }
+ */ + public static Row row(Object... values) { + return Row.of(values); + } + + /** + * Creates array with specified values. + *

This method is used to define row in convenient way such as: + * + *

 {@code
+   * row(1, array("abc1", "def1"), map("abc1", 1, "def1", 3), row(1, "abc1"))
+   * }
+ */ + @SafeVarargs + public static T[] array(T... values) { + return values; + } + + /** + * Creates map with specified keys and values. + *

This method is used to define row in convenient way such as: + * + *

 {@code
+   * row(1, array("abc1", "def1"), map("abc1", 1, "def1", 3), row(1, "abc1"))
+   * }
+ * + *

NOTE: be careful of the order of keys and values. If you make + * a mistake, {@link ClassCastException} will occur later than the + * creation of the map due to type erasure. + */ + public static Map map(Object... kwargs) { + HashMap map = new HashMap<>(); + for (int i = 0; i < kwargs.length; i += 2) { + map.put(kwargs[i], kwargs[i + 1]); + } + @SuppressWarnings("unchecked") + Map resultMap = (Map) map; + return resultMap; + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java index 2830eefef013f..5a9fd802bf040 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java @@ -19,12 +19,11 @@ package org.apache.hudi.utils.source; import org.apache.hudi.adapter.DataStreamScanProviderAdapter; +import org.apache.hudi.util.JsonDeserializationFunction; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; -import org.apache.flink.formats.common.TimestampFormat; -import org.apache.flink.formats.json.JsonRowDataDeserializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -37,7 +36,6 @@ import org.apache.flink.table.types.logical.RowType; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; @@ -49,7 +47,7 @@ * A continuous file source that can trigger checkpoints continuously. * *

It loads the data in the specified file and split the data into number of checkpoints batches. - * Say, if you want 4 checkpoints and there are 8 records in the file, the emit strategy is: + * Say, if you want 4 checkpoints and there are 8 records in the file, the emission strategy is: * *

  *   | 2 records | 2 records | 2 records | 2 records |
@@ -57,6 +55,8 @@
  * 
* *

If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source. + * + *

NOTE: this class is represented twice: in test utils and in quickstart. Don't forget to update both files. */ public class ContinuousFileSource implements ScanTableSource { @@ -85,18 +85,10 @@ public boolean isBounded() { @Override public DataStream produceDataStream(StreamExecutionEnvironment execEnv) { final RowType rowType = (RowType) tableSchema.toSourceRowDataType().getLogicalType(); - JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( - rowType, - InternalTypeInfo.of(rowType), - false, - true, - TimestampFormat.ISO_8601); - return execEnv.addSource(new BoundedSourceFunction(path, conf.getInteger(CHECKPOINTS))) .name("continuous_file_source") .setParallelism(1) - .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)), - InternalTypeInfo.of(rowType)); + .map(JsonDeserializationFunction.getInstance(rowType), InternalTypeInfo.of(rowType)); } }; } diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index e47fb6036210c..6922ada9acf16 100644 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -23,6 +23,7 @@ import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.vector.ColumnVector; import org.apache.flink.table.data.vector.writable.WritableColumnVector; import org.apache.flink.table.types.logical.LogicalType; @@ -266,7 +267,7 @@ public boolean reachedEnd() throws IOException { return !ensureBatch(); } - public ColumnarRowData nextRecord() { + public RowData nextRecord() { // return the next row row.setRowId(this.nextRow++); return row; diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index 92f5d1e191771..1826d5bea4c76 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -22,6 +22,7 @@ import org.apache.flink.formats.parquet.vector.reader.ColumnReader; import org.apache.flink.table.data.ColumnarRowData; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.vector.ColumnVector; import org.apache.flink.table.data.vector.VectorizedColumnBatch; import org.apache.flink.table.data.vector.writable.WritableColumnVector; @@ -266,7 +267,7 @@ public boolean reachedEnd() throws IOException { return !ensureBatch(); } - public ColumnarRowData nextRecord() { + public RowData nextRecord() { // return the next row row.setRowId(this.nextRow++); return row; diff --git a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index cd1bb59c34d40..1872ec385b4a9 100644 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -21,6 +21,7 @@ import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.columnar.ColumnarRowData; import org.apache.flink.table.data.columnar.vector.ColumnVector; import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; @@ -266,7 +267,7 @@ public boolean reachedEnd() throws IOException { return !ensureBatch(); } - public ColumnarRowData nextRecord() { + public RowData nextRecord() { // return the next row row.setRowId(this.nextRow++); return row; diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java index b46cd9a9f81f7..52b3d3b74e5e4 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java @@ -20,14 +20,13 @@ package org.apache.hudi.gcp.bigquery; import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.sync.common.HoodieSyncConfig; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParametersDelegate; import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; import java.util.Properties; /** @@ -101,38 +100,27 @@ public static class BigQuerySyncConfigParams { public String datasetName; @Parameter(names = {"--dataset-location"}, description = "Location of the target dataset in BigQuery", required = true) public String datasetLocation; - @Parameter(names = {"--table-name"}, description = "Name of the target table in BigQuery", required = true) - public String tableName; @Parameter(names = {"--source-uri"}, description = "Name of the source uri gcs path of the table", required = true) public String sourceUri; @Parameter(names = {"--source-uri-prefix"}, description = "Name of the source uri gcs path prefix of the table", required = true) public String sourceUriPrefix; - @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) - public String basePath; - @Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.") - public List partitionFields = new ArrayList<>(); - @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") - public boolean useFileListingFromMetadata = false; - @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" - + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") - public boolean assumeDatePartitioning = false; public boolean isHelp() { return hoodieSyncConfigParams.isHelp(); } - public Properties toProps() { - final Properties props = hoodieSyncConfigParams.toProps(); - props.setProperty(BIGQUERY_SYNC_PROJECT_ID.key(), projectId); - props.setProperty(BIGQUERY_SYNC_DATASET_NAME.key(), datasetName); - props.setProperty(BIGQUERY_SYNC_DATASET_LOCATION.key(), datasetLocation); - props.setProperty(BIGQUERY_SYNC_TABLE_NAME.key(), tableName); - props.setProperty(BIGQUERY_SYNC_SOURCE_URI.key(), sourceUri); - props.setProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), sourceUriPrefix); - props.setProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), basePath); - props.setProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key(), String.join(",", partitionFields)); - props.setProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(useFileListingFromMetadata)); - props.setProperty(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), String.valueOf(assumeDatePartitioning)); + public TypedProperties toProps() { + final TypedProperties props = hoodieSyncConfigParams.toProps(); + props.setPropertyIfNonNull(BIGQUERY_SYNC_PROJECT_ID.key(), projectId); + props.setPropertyIfNonNull(BIGQUERY_SYNC_DATASET_NAME.key(), datasetName); + props.setPropertyIfNonNull(BIGQUERY_SYNC_DATASET_LOCATION.key(), datasetLocation); + props.setPropertyIfNonNull(BIGQUERY_SYNC_TABLE_NAME.key(), hoodieSyncConfigParams.tableName); + props.setPropertyIfNonNull(BIGQUERY_SYNC_SOURCE_URI.key(), sourceUri); + props.setPropertyIfNonNull(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), sourceUriPrefix); + props.setPropertyIfNonNull(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), hoodieSyncConfigParams.basePath); + props.setPropertyIfNonNull(BIGQUERY_SYNC_PARTITION_FIELDS.key(), String.join(",", hoodieSyncConfigParams.partitionFields)); + props.setPropertyIfNonNull(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), hoodieSyncConfigParams.useFileListingFromMetadata); + props.setPropertyIfNonNull(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), hoodieSyncConfigParams.assumeDatePartitioning); return props; } } diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java new file mode 100644 index 0000000000000..898358484d979 --- /dev/null +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncToolArgs.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.gcp.bigquery; + +import com.beust.jcommander.JCommander; +import org.junit.jupiter.api.Test; + +import java.util.Properties; + +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI_PREFIX; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SYNC_BASE_PATH; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_TABLE_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestBigQuerySyncToolArgs { + + @Test + public void testArgsParse() { + BigQuerySyncConfig.BigQuerySyncConfigParams params = new BigQuerySyncConfig.BigQuerySyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + String[] args = { + "--project-id", "hudi-bq", + "--dataset-name", "foobar", + "--dataset-location", "us-west1", + "--table", "foobartable", + "--source-uri", "gs://foobartable/year=*", + "--source-uri-prefix", "gs://foobartable/", + "--base-path", "gs://foobartable", + "--partitioned-by", "year,month,day", + "--use-file-listing-from-metadata" + }; + cmd.parse(args); + + final Properties props = params.toProps(); + assertEquals("hudi-bq", props.getProperty(BIGQUERY_SYNC_PROJECT_ID.key())); + assertEquals("foobar", props.getProperty(BIGQUERY_SYNC_DATASET_NAME.key())); + assertEquals("us-west1", props.getProperty(BIGQUERY_SYNC_DATASET_LOCATION.key())); + assertEquals("foobartable", props.getProperty(BIGQUERY_SYNC_TABLE_NAME.key())); + assertEquals("gs://foobartable/year=*", props.getProperty(BIGQUERY_SYNC_SOURCE_URI.key())); + assertEquals("gs://foobartable/", props.getProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key())); + assertEquals("gs://foobartable", props.getProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key())); + assertEquals("year,month,day", props.getProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key())); + assertEquals("true", props.getProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key())); + assertFalse(props.containsKey(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key())); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index ce441bf2e2892..ed9e16c8d0a7c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -20,28 +20,19 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableQueryType; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.hudi.metadata.HoodieTableMetadataUtil; -import org.apache.avro.Schema; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -64,17 +55,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.stream.Collectors; -import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS; -import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE; -import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE; -import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.buildMetadataConfig; -import static org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getFileStatus; +import static org.apache.hudi.common.util.ValidationUtils.checkState; /** * Base implementation of the Hive's {@link FileInputFormat} allowing for reading of Hudi's @@ -200,7 +186,7 @@ protected List listStatusForIncrementalMode(JobConf job, return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get()); } - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, Option instantOpt, String basePath, Option virtualKeyInfoOpt) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient metaClient) { Option baseFileOpt = fileSlice.getBaseFile(); if (baseFileOpt.isPresent()) { @@ -225,7 +211,7 @@ private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus @Nonnull private List listStatusForSnapshotMode(JobConf job, Map tableMetaClientMap, - List snapshotPaths) throws IOException { + List snapshotPaths) { HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job); List targetFiles = new ArrayList<>(); @@ -233,7 +219,6 @@ private List listStatusForSnapshotMode(JobConf job, Map> groupedPaths = HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths); - Map fsViewCache = new HashMap<>(); for (Map.Entry> entry : groupedPaths.entrySet()) { HoodieTableMetaClient tableMetaClient = entry.getKey(); @@ -247,83 +232,31 @@ private List listStatusForSnapshotMode(JobConf job, boolean shouldIncludePendingCommits = HoodieHiveUtils.shouldIncludePendingCommits(job, tableMetaClient.getTableConfig().getTableName()); - // NOTE: Fetching virtual key info is a costly operation as it needs to load the commit metadata. - // This is only needed for MOR realtime splits. Hence, for COW tables, this can be avoided. - Option virtualKeyInfoOpt = tableMetaClient.getTableType().equals(COPY_ON_WRITE) ? Option.empty() : getHoodieVirtualKeyInfo(tableMetaClient); - String basePath = tableMetaClient.getBasePathV2().toString(); - - if (conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS) && HoodieTableMetadataUtil.isFilesPartitionAvailable(tableMetaClient)) { - HiveHoodieTableFileIndex fileIndex = - new HiveHoodieTableFileIndex( - engineContext, - tableMetaClient, - props, - HoodieTableQueryType.SNAPSHOT, - partitionPaths, - queryCommitInstant, - shouldIncludePendingCommits); - - Map> partitionedFileSlices = fileIndex.listFileSlices(); - - targetFiles.addAll( - partitionedFileSlices.values() - .stream() - .flatMap(Collection::stream) - .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) - .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex.getLatestCompletedInstant(), basePath, virtualKeyInfoOpt)) - .collect(Collectors.toList()) - ); - } else { - HoodieTimeline timeline = getActiveTimeline(tableMetaClient, shouldIncludePendingCommits); - Option queryInstant = queryCommitInstant.or(() -> timeline.lastInstant().map(HoodieInstant::getTimestamp)); - validateInstant(timeline, queryInstant); - - try { - HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(tableMetaClient, hoodieTableMetaClient -> - FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, hoodieTableMetaClient, buildMetadataConfig(job), timeline)); - - List filteredFileSlices = new ArrayList<>(); - - for (Path p : entry.getValue()) { - String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), p); - - List fileSlices = queryInstant.map( - instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relativePartitionPath, instant)) - .orElse(fsView.getLatestFileSlices(relativePartitionPath)) - .collect(Collectors.toList()); - - filteredFileSlices.addAll(fileSlices); - } - - targetFiles.addAll( - filteredFileSlices.stream() - .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) - .map(fileSlice -> createFileStatusUnchecked(fileSlice, timeline.filterCompletedInstants().lastInstant(), basePath, virtualKeyInfoOpt)) - .collect(Collectors.toList())); - } finally { - fsViewCache.forEach(((metaClient, fsView) -> fsView.close())); - } - } + HiveHoodieTableFileIndex fileIndex = + new HiveHoodieTableFileIndex( + engineContext, + tableMetaClient, + props, + HoodieTableQueryType.SNAPSHOT, + partitionPaths, + queryCommitInstant, + shouldIncludePendingCommits); + + Map> partitionedFileSlices = fileIndex.listFileSlices(); + + targetFiles.addAll( + partitionedFileSlices.values() + .stream() + .flatMap(Collection::stream) + .filter(fileSlice -> checkIfValidFileSlice(fileSlice)) + .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex, tableMetaClient)) + .collect(Collectors.toList()) + ); } return targetFiles; } - private static HoodieTimeline getActiveTimeline(HoodieTableMetaClient metaClient, boolean shouldIncludePendingCommits) { - HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline(); - if (shouldIncludePendingCommits) { - return timeline; - } else { - return timeline.filterCompletedAndCompactionInstants(); - } - } - - private static void validateInstant(HoodieTimeline activeTimeline, Option queryInstant) { - if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) { - throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get())); - } - } - protected boolean checkIfValidFileSlice(FileSlice fileSlice) { Option baseFileOpt = fileSlice.getBaseFile(); Option latestLogFileOpt = fileSlice.getLatestLogFile(); @@ -338,32 +271,17 @@ protected boolean checkIfValidFileSlice(FileSlice fileSlice) { } } + private void validate(List targetFiles, List legacyFileStatuses) { + List diff = CollectionUtils.diff(targetFiles, legacyFileStatuses); + checkState(diff.isEmpty(), "Should be empty"); + } + @Nonnull protected static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { try { - return getFileStatus(baseFile); + return HoodieInputFormatUtils.getFileStatus(baseFile); } catch (IOException ioe) { throw new HoodieIOException("Failed to get file-status", ioe); } } - - protected static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { - HoodieTableConfig tableConfig = metaClient.getTableConfig(); - if (tableConfig.populateMetaFields()) { - return Option.empty(); - } - TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); - try { - Schema schema = tableSchemaResolver.getTableAvroSchema(); - boolean isNonPartitionedKeyGen = StringUtils.isNullOrEmpty(tableConfig.getPartitionFieldProp()); - return Option.of( - new HoodieVirtualKeyInfo( - tableConfig.getRecordKeyFieldProp(), - isNonPartitionedKeyGen ? Option.empty() : Option.of(tableConfig.getPartitionFieldProp()), - schema.getField(tableConfig.getRecordKeyFieldProp()).pos(), - isNonPartitionedKeyGen ? Option.empty() : Option.of(schema.getField(tableConfig.getPartitionFieldProp()).pos()))); - } catch (Exception exception) { - throw new HoodieException("Fetching table schema failed with exception ", exception); - } - } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 448a5811bcc11..2799f343b9ebf 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -18,36 +18,42 @@ package org.apache.hudi.hadoop.realtime; -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.SplitLocationInfo; -import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; +import org.apache.hudi.hadoop.HiveHoodieTableFileIndex; import org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.SplitLocationInfo; +import org.apache.hadoop.mapreduce.Job; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -84,19 +90,19 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { } @Override - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, - Option latestCompletedInstantOpt, - String tableBasePath, - Option virtualKeyInfoOpt) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient metaClient) { Option baseFileOpt = fileSlice.getBaseFile(); Option latestLogFileOpt = fileSlice.getLatestLogFile(); Stream logFiles = fileSlice.getLogFiles(); + Option latestCompletedInstantOpt = fileIndex.getLatestCompletedInstant(); + String tableBasePath = fileIndex.getBasePath().toString(); + // Check if we're reading a MOR table if (baseFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, virtualKeyInfoOpt); + return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, getHoodieVirtualKeyInfo(metaClient)); } else if (latestLogFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, virtualKeyInfoOpt); + return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, getHoodieVirtualKeyInfo(metaClient)); } else { throw new IllegalStateException("Invalid state: either base-file or log-file has to be present"); } @@ -382,5 +388,24 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFil throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); } } -} + private static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + if (tableConfig.populateMetaFields()) { + return Option.empty(); + } + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); + try { + Schema schema = tableSchemaResolver.getTableAvroSchema(); + boolean isNonPartitionedKeyGen = StringUtils.isNullOrEmpty(tableConfig.getPartitionFieldProp()); + return Option.of( + new HoodieVirtualKeyInfo( + tableConfig.getRecordKeyFieldProp(), + isNonPartitionedKeyGen ? Option.empty() : Option.of(tableConfig.getPartitionFieldProp()), + schema.getField(tableConfig.getRecordKeyFieldProp()).pos(), + isNonPartitionedKeyGen ? Option.empty() : Option.of(schema.getField(tableConfig.getPartitionFieldProp()).pos()))); + } catch (Exception exception) { + throw new HoodieException("Fetching table schema failed with exception ", exception); + } + } +} 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 fa1b60480651d..7b47ffa75f3d2 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 @@ -450,7 +450,7 @@ public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) { * @param dataFile * @return */ - public static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFile dataFile) { + private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFile dataFile) { Path dataPath = dataFile.getFileStatus().getPath(); try { if (dataFile.getFileSize() == 0) { diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index 0def4acead001..616f2a6e5c2c6 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -51,16 +51,6 @@ test - - - - org.eclipse.jetty.aggregate - jetty-all - ${jetty.version} - uber - test - - org.apache.spark spark-sql_${scala.binary.version} diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java index 76f9d7424ac90..27733bb3804c5 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieContinousTestSuiteWriter.java @@ -85,8 +85,14 @@ public HoodieContinousTestSuiteWriter(JavaSparkContext jsc, Properties props, Ho @Override public void shutdownResources() { - log.info("Shutting down deltastreamer gracefully "); - this.deltaStreamerWrapper.shutdownGracefully(); + if (this.deltaStreamerWrapper != null) { + log.info("Shutting down DS wrapper gracefully "); + this.deltaStreamerWrapper.shutdownGracefully(); + } + if (this.writeClient != null) { + log.info("Closing local write client"); + this.writeClient.close(); + } } @Override diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java index 91a7cf358c011..46f793ef12bf6 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieInlineTestSuiteWriter.java @@ -65,7 +65,14 @@ public HoodieInlineTestSuiteWriter(JavaSparkContext jsc, Properties props, Hoodi } public void shutdownResources() { - // no-op for non continuous mode test suite writer. + if (this.deltaStreamerWrapper != null) { + log.info("Shutting down DS wrapper gracefully "); + this.deltaStreamerWrapper.shutdownGracefully(); + } + if (this.writeClient != null) { + log.info("Closing local write client"); + this.writeClient.close(); + } } public RDD getNextBatch() throws Exception { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 242d7eb86796f..067c14ef3ec27 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hbase.io.hfile.CacheConfig import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieBaseRelation._ +import org.apache.hudi.AvroConversionUtils.getAvroSchemaWithDefaults import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.utils.SparkInternalSchemaConverter @@ -41,6 +42,7 @@ import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema} import org.apache.hudi.io.storage.HoodieHFileReader +import org.apache.hudi.metadata.HoodieTableMetadata import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -94,6 +96,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, protected val sparkSession: SparkSession = sqlContext.sparkSession + protected def tableName: String = metaClient.getTableConfig.getTableName + protected lazy val conf: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) protected lazy val jobConf = new JobConf(conf) @@ -150,12 +154,13 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, } } + val (name, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) val avroSchema = internalSchemaOpt.map { is => - AvroInternalSchemaConverter.convert(is, "schema") + AvroInternalSchemaConverter.convert(is, namespace + "." + name) } orElse { specifiedQueryTimestamp.map(schemaResolver.getTableAvroSchema) } orElse { - schemaSpec.map(convertToAvroSchema) + schemaSpec.map(s => convertToAvroSchema(s, tableName)) } getOrElse { Try(schemaResolver.getTableAvroSchema) match { case Success(schema) => schema @@ -278,7 +283,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, * Determines whether relation's schema could be pruned by Spark's Optimizer */ def canPruneRelationSchema: Boolean = - (fileFormat.isInstanceOf[ParquetFileFormat] || fileFormat.isInstanceOf[OrcFileFormat]) && + !HoodieTableMetadata.isMetadataTable(basePath.toString) && (fileFormat.isInstanceOf[ParquetFileFormat] || fileFormat.isInstanceOf[OrcFileFormat]) && // NOTE: Some relations might be disabling sophisticated schema pruning techniques (for ex, nested schema pruning) // TODO(HUDI-XXX) internal schema doesn't support nested schema pruning currently !hasSchemaOnRead @@ -326,7 +331,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, // schema conversion, which is lossy in nature (for ex, it doesn't preserve original Avro type-names) and // could have an effect on subsequent de-/serializing records in some exotic scenarios (when Avro unions // w/ more than 2 types are involved) - val sourceSchema = optimizerPrunedDataSchema.map(convertToAvroSchema).getOrElse(tableAvroSchema) + val sourceSchema = optimizerPrunedDataSchema.map(s => convertToAvroSchema(s, tableName)).getOrElse(tableAvroSchema) val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) = projectSchema(Either.cond(internalSchemaOpt.isDefined, internalSchemaOpt.get, sourceSchema), targetColumns) @@ -626,8 +631,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, val prunedRequiredSchema = prunePartitionColumns(requiredSchema.structTypeSchema) (partitionSchema, - HoodieTableSchema(prunedDataStructSchema, convertToAvroSchema(prunedDataStructSchema).toString), - HoodieTableSchema(prunedRequiredSchema, convertToAvroSchema(prunedRequiredSchema).toString)) + HoodieTableSchema(prunedDataStructSchema, convertToAvroSchema(prunedDataStructSchema, tableName).toString), + HoodieTableSchema(prunedRequiredSchema, convertToAvroSchema(prunedRequiredSchema, tableName).toString)) } else { (StructType(Nil), tableSchema, requiredSchema) } @@ -646,8 +651,11 @@ object HoodieBaseRelation extends SparkAdapterSupport { def generateUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = HoodieCatalystExpressionUtils.generateUnsafeProjection(from, to) - def convertToAvroSchema(structSchema: StructType): Schema = - sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, "Record") + def convertToAvroSchema(structSchema: StructType, tableName: String): Schema = { + val (recordName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) + val avroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, recordName, namespace) + getAvroSchemaWithDefaults(avroSchema, structSchema) + } def getPartitionPath(fileStatus: FileStatus): Path = fileStatus.getPath.getParent diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index c9f424825031f..411fa98c055fa 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -28,7 +28,6 @@ import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieConfig, HoodieMetadataConfig, TypedProperties} -import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model._ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} @@ -55,6 +54,7 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.{SPARK_VERSION, SparkContext} @@ -232,8 +232,11 @@ object HoodieSparkSqlWriter { genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect() } // Create a HoodieWriteClient & issue the delete. + val tableMetaClient = HoodieTableMetaClient.builder + .setConf(sparkContext.hadoopConfiguration).setBasePath(basePath.toString).build() + val schemaStr = new TableSchemaResolver(tableMetaClient).getTableAvroSchema.toString val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, - null, path, tblName, + schemaStr, path, tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key))) .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] // Issue delete partitions @@ -250,7 +253,8 @@ object HoodieSparkSqlWriter { // TODO(HUDI-4472) revisit and simplify schema handling val sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace) - val latestTableSchema = getLatestTableSchema(sqlContext.sparkSession, tableMetaClient).getOrElse(sourceSchema) + val tableIdentifier = TableIdentifier(tblName, if (databaseName.isEmpty) None else Some(databaseName)) + val latestTableSchema = getLatestTableSchema(sqlContext.sparkSession, tableIdentifier, tableMetaClient).getOrElse(sourceSchema) val schemaEvolutionEnabled = parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), "false").toBoolean var internalSchemaOpt = getLatestTableInternalSchema(hoodieConfig, tableMetaClient) @@ -339,12 +343,21 @@ object HoodieSparkSqlWriter { } // Check for errors and commit the write. + try { val (writeSuccessful, compactionInstant, clusteringInstant) = commitAndPerformPostOperations(sqlContext.sparkSession, df.schema, writeResult, parameters, writeClient, tableConfig, jsc, TableInstantInfo(basePath, instantTime, commitActionType, operation)) - - (writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, clusteringInstant, writeClient, tableConfig) + (writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, clusteringInstant, writeClient, tableConfig) + } finally { + // close the write client in all cases + val asyncCompactionEnabled = isAsyncCompactionEnabled(writeClient, tableConfig, parameters, jsc.hadoopConfiguration()) + val asyncClusteringEnabled = isAsyncClusteringEnabled(writeClient, parameters) + if (!asyncCompactionEnabled && !asyncClusteringEnabled) { + log.info("Closing write client") + writeClient.close() + } + } } } @@ -394,9 +407,24 @@ object HoodieSparkSqlWriter { } private def getLatestTableSchema(spark: SparkSession, + tableId: TableIdentifier, tableMetaClient: HoodieTableMetaClient): Option[Schema] = { val tableSchemaResolver = new TableSchemaResolver(tableMetaClient) - toScalaOption(tableSchemaResolver.getTableAvroSchemaFromLatestCommit(false)) + val latestTableSchemaFromCommitMetadata = toScalaOption(tableSchemaResolver.getTableAvroSchemaFromLatestCommit(false)) + latestTableSchemaFromCommitMetadata.orElse { + getCatalogTable(spark, tableId).map { catalogTable => + val (structName, namespace) = getAvroRecordNameAndNamespace(tableId.table) + convertStructTypeToAvroSchema(catalogTable.schema, structName, namespace) + } + } + } + + private def getCatalogTable(spark: SparkSession, tableId: TableIdentifier): Option[CatalogTable] = { + if (spark.sessionState.catalog.tableExists(tableId)) { + Some(spark.sessionState.catalog.getTableMetadata(tableId)) + } else { + None + } } def registerKryoClassesAndGetGenericRecords(tblName: String, sparkContext: SparkContext, df: Dataset[Row], @@ -731,9 +759,6 @@ object HoodieSparkSqlWriter { tableInstantInfo.basePath, schema) log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled") - if (!asyncCompactionEnabled && !asyncClusteringEnabled) { - client.close() - } (commitSuccess && metaSyncSuccess, compactionInstant, clusteringInstant) } else { log.error(s"${tableInstantInfo.operation} failed with errors") diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 335fe68bd2099..1af76d46fd936 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -72,7 +72,6 @@ object HoodieWriterUtils { hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT) - hoodieConfig.setDefaultValue(HiveSyncConfigHolder.METASTORE_URIS) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USER) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_PASS) hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_URL) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 1d0d533e5bb81..e1f8d1c51a36d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -165,7 +165,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, StructType(requiredDataSchema.structTypeSchema.fields .filterNot(f => superfluousColumnNames.contains(f.name))) - HoodieTableSchema(prunedStructSchema, convertToAvroSchema(prunedStructSchema).toString) + HoodieTableSchema(prunedStructSchema, convertToAvroSchema(prunedStructSchema, tableName).toString) } val requiredSchemaReaderSkipMerging = createBaseFileReader( diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala index c31cd3b20565e..daabbf93ab166 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala @@ -22,14 +22,15 @@ import org.apache.hudi.HoodieWriterUtils._ import org.apache.hudi.common.config.DFSPropertiesConfiguration import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} -import org.apache.hudi.common.util.{StringUtils, ValidationUtils} +import org.apache.hudi.common.util.StringUtils +import org.apache.hudi.common.util.ValidationUtils.checkArgument import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.{AvroConversionUtils, DataSourceOptionsHelper} import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.hudi.HoodieOptionConfig -import org.apache.spark.sql.hudi.HoodieOptionConfig.SQL_KEY_TABLE_PRIMARY_KEY +import org.apache.spark.sql.hudi.HoodieOptionConfig._ import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.{AnalysisException, SparkSession} @@ -215,20 +216,21 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten private def parseSchemaAndConfigs(): (StructType, Map[String, String]) = { val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap val globalTableConfigs = mappingSparkDatasourceConfigsToTableConfigs(globalProps) - val globalSqlOptions = HoodieOptionConfig.mappingTableConfigToSqlOption(globalTableConfigs) + val globalSqlOptions = mapTableConfigsToSqlOptions(globalTableConfigs) - val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(globalSqlOptions ++ catalogProperties) + val sqlOptions = withDefaultSqlOptions(globalSqlOptions ++ + mapDataSourceWriteOptionsToSqlOptions(catalogProperties) ++ catalogProperties) // get final schema and parameters val (finalSchema, tableConfigs) = (table.tableType, hoodieTableExists) match { case (CatalogTableType.EXTERNAL, true) => val existingTableConfig = tableConfig.getProps.asScala.toMap val currentTableConfig = globalTableConfigs ++ existingTableConfig - val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(catalogProperties) + val catalogTableProps = mapSqlOptionsToTableConfigs(catalogProperties) validateTableConfig(spark, catalogTableProps, convertMapToHoodieConfig(existingTableConfig)) val options = extraTableConfig(hoodieTableExists, currentTableConfig) ++ - HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) ++ currentTableConfig + mapSqlOptionsToTableConfigs(sqlOptions) ++ currentTableConfig val schemaFromMetaOpt = loadTableSchemaByMetaClient() val schema = if (schemaFromMetaOpt.nonEmpty) { @@ -242,11 +244,11 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten (schema, options) case (_, false) => - ValidationUtils.checkArgument(table.schema.nonEmpty, + checkArgument(table.schema.nonEmpty, s"Missing schema for Create Table: $catalogTableName") val schema = table.schema val options = extraTableConfig(tableExists = false, globalTableConfigs) ++ - HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) + mapSqlOptionsToTableConfigs(sqlOptions) (addMetaFields(schema), options) case (CatalogTableType.MANAGED, true) => @@ -254,7 +256,7 @@ class HoodieCatalogTable(val spark: SparkSession, var table: CatalogTable) exten s". The associated location('$tableLocation') already exists.") } HoodieOptionConfig.validateTable(spark, finalSchema, - HoodieOptionConfig.mappingTableConfigToSqlOption(tableConfigs)) + mapTableConfigsToSqlOptions(tableConfigs)) val resolver = spark.sessionState.conf.resolver val dataSchema = finalSchema.filterNot { f => diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala new file mode 100644 index 0000000000000..ce1a719cb94ba --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetFileFormatHelper.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.parquet.hadoop.metadata.FileMetaData +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} + +object HoodieParquetFileFormatHelper { + + def buildImplicitSchemaChangeInfo(hadoopConf: Configuration, + parquetFileMetaData: FileMetaData, + requiredSchema: StructType): (java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]], StructType) = { + val implicitTypeChangeInfo: java.util.Map[Integer, org.apache.hudi.common.util.collection.Pair[DataType, DataType]] = new java.util.HashMap() + val convert = new ParquetToSparkSchemaConverter(hadoopConf) + val fileStruct = convert.convert(parquetFileMetaData.getSchema) + val fileStructMap = fileStruct.fields.map(f => (f.name, f.dataType)).toMap + val sparkRequestStructFields = requiredSchema.map(f => { + val requiredType = f.dataType + if (fileStructMap.contains(f.name) && !isDataTypeEqual(requiredType, fileStructMap(f.name))) { + implicitTypeChangeInfo.put(new Integer(requiredSchema.fieldIndex(f.name)), org.apache.hudi.common.util.collection.Pair.of(requiredType, fileStructMap(f.name))) + StructField(f.name, fileStructMap(f.name), f.nullable) + } else { + f + } + }) + (implicitTypeChangeInfo, StructType(sparkRequestStructFields)) + } + + def isDataTypeEqual(requiredType: DataType, fileType: DataType): Boolean = (requiredType, fileType) match { + case (requiredType, fileType) if requiredType == fileType => true + + case (ArrayType(rt, _), ArrayType(ft, _)) => + // Do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(rt, ft) + + case (MapType(requiredKey, requiredValue, _), MapType(fileKey, fileValue, _)) => + // Likewise, do not care about nullability as schema evolution require fields to be nullable + isDataTypeEqual(requiredKey, fileKey) && isDataTypeEqual(requiredValue, fileValue) + + case (StructType(requiredFields), StructType(fileFields)) => + // Find fields that are in requiredFields and fileFields as they might not be the same during add column + change column operations + val commonFieldNames = requiredFields.map(_.name) intersect fileFields.map(_.name) + + // Need to match by name instead of StructField as name will stay the same whilst type may change + val fileFilteredFields = fileFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + val requiredFilteredFields = requiredFields.filter(f => commonFieldNames.contains(f.name)).sortWith(_.name < _.name) + + // Sorting ensures that the same field names are being compared for type differences + requiredFilteredFields.zip(fileFilteredFields).forall { + case (requiredField, fileFilteredField) => + isDataTypeEqual(requiredField.dataType, fileFilteredField.dataType) + } + + case _ => false + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala index 732367cf5a5e5..ebddf73f7b121 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala @@ -72,7 +72,7 @@ object HoodieOptionConfig { /** * The mapping of the sql short name key to the hoodie's config key. */ - private lazy val keyMapping: Map[String, String] = { + private lazy val sqlOptionKeyToWriteConfigKey: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields .filter(f => f.getType == classOf[HoodieSQLOption[_]]) .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) @@ -80,11 +80,14 @@ object HoodieOptionConfig { .toMap } + private lazy val writeConfigKeyToSqlOptionKey: Map[String, String] = + sqlOptionKeyToWriteConfigKey.map(f => f._2 -> f._1) + /** * The mapping of the sql short name key to the hoodie table config key * defined in HoodieTableConfig. */ - private lazy val keyTableConfigMapping: Map[String, String] = { + private lazy val sqlOptionKeyToTableConfigKey: Map[String, String] = { HoodieOptionConfig.getClass.getDeclaredFields .filter(f => f.getType == classOf[HoodieSQLOption[_]]) .map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]}) @@ -93,41 +96,43 @@ object HoodieOptionConfig { .toMap } - private lazy val tableConfigKeyToSqlKey: Map[String, String] = - keyTableConfigMapping.map(f => f._2 -> f._1) + private lazy val tableConfigKeyToSqlOptionKey: Map[String, String] = + sqlOptionKeyToTableConfigKey.map(f => f._2 -> f._1) /** * Mapping of the short sql value to the hoodie's config value */ - private val valueMapping: Map[String, String] = Map ( + private val sqlOptionValueToWriteConfigValue: Map[String, String] = Map ( SQL_VALUE_TABLE_TYPE_COW -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, SQL_VALUE_TABLE_TYPE_MOR -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL ) - private lazy val reverseValueMapping = valueMapping.map(f => f._2 -> f._1) + private lazy val writeConfigValueToSqlOptionValue = sqlOptionValueToWriteConfigValue.map(f => f._2 -> f._1) def withDefaultSqlOptions(options: Map[String, String]): Map[String, String] = defaultSqlOptions ++ options /** - * Mapping the sql's short name key/value in the options to the hoodie's config key/value. - * @param options - * @return + * Map SQL options to data source write configs. */ - def mappingSqlOptionToHoodieParam(options: Map[String, String]): Map[String, String] = { + def mapSqlOptionsToDataSourceWriteConfigs(options: Map[String, String]): Map[String, String] = { options.map (kv => - keyMapping.getOrElse(kv._1, kv._1) -> valueMapping.getOrElse(kv._2, kv._2)) + sqlOptionKeyToWriteConfigKey.getOrElse(kv._1, kv._1) -> sqlOptionValueToWriteConfigValue.getOrElse(kv._2, kv._2)) } /** - * Mapping the sql options to the hoodie table config which used to store to the hoodie - * .properties when create the table. - * @param options - * @return + * Mapping the data source write configs to SQL options. */ - def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = { + def mapDataSourceWriteOptionsToSqlOptions(options: Map[String, String]): Map[String, String] = { + options.map(kv => writeConfigKeyToSqlOptionKey.getOrElse(kv._1, kv._1) -> writeConfigValueToSqlOptionValue.getOrElse(kv._2, kv._2)) + } + + /** + * Map SQL options to table configs. + */ + def mapSqlOptionsToTableConfigs(options: Map[String, String]): Map[String, String] = { options.map { case (k, v) => - if (keyTableConfigMapping.contains(k)) { - keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v) + if (sqlOptionKeyToTableConfigKey.contains(k)) { + sqlOptionKeyToTableConfigKey(k) -> sqlOptionValueToWriteConfigValue.getOrElse(v, v) } else { k -> v } @@ -135,10 +140,10 @@ object HoodieOptionConfig { } /** - * Mapping the table config (loaded from the hoodie.properties) to the sql options. + * Map table configs to SQL options. */ - def mappingTableConfigToSqlOption(options: Map[String, String]): Map[String, String] = { - options.map(kv => tableConfigKeyToSqlKey.getOrElse(kv._1, kv._1) -> reverseValueMapping.getOrElse(kv._2, kv._2)) + def mapTableConfigsToSqlOptions(options: Map[String, String]): Map[String, String] = { + options.map(kv => tableConfigKeyToSqlOptionKey.getOrElse(kv._1, kv._1) -> writeConfigValueToSqlOptionValue.getOrElse(kv._2, kv._2)) } val defaultSqlOptions: Map[String, String] = { @@ -150,42 +155,30 @@ object HoodieOptionConfig { .toMap } - /** - * Get the primary key from the table options. - * @param options - * @return - */ - def getPrimaryColumns(options: Map[String, String]): Array[String] = { - val params = mappingSqlOptionToHoodieParam(options) - params.get(DataSourceWriteOptions.RECORDKEY_FIELD.key) - .map(_.split(",").filter(_.nonEmpty)) - .getOrElse(Array.empty) - } - /** * Get the table type from the table options. * @param options * @return */ def getTableType(options: Map[String, String]): String = { - val params = mappingSqlOptionToHoodieParam(options) + val params = mapSqlOptionsToDataSourceWriteConfigs(options) params.getOrElse(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.TABLE_TYPE.defaultValue) } def getPreCombineField(options: Map[String, String]): Option[String] = { - val params = mappingSqlOptionToHoodieParam(options) + val params = mapSqlOptionsToDataSourceWriteConfigs(options) params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty) } def deleteHoodieOptions(options: Map[String, String]): Map[String, String] = { - options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => keyMapping.contains(kv._1)) + options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => sqlOptionKeyToWriteConfigKey.contains(kv._1)) } // extract primaryKey, preCombineField, type options def extractSqlOptions(options: Map[String, String]): Map[String, String] = { - val sqlOptions = mappingTableConfigToSqlOption(options) - val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) + val sqlOptions = mapTableConfigsToSqlOptions(options) + val targetOptions = sqlOptionKeyToWriteConfigKey.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName) sqlOptions.filterKeys(targetOptions.contains) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala index 025a224373aed..abc766308288c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala @@ -249,7 +249,7 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport { def withSparkConf(spark: SparkSession, options: Map[String, String]) (baseConfig: Map[String, String] = Map.empty): Map[String, String] = { baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority - (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options)) + (spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mapSqlOptionsToDataSourceWriteConfigs(options)) .filterKeys(isHoodieConfigKey) } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 61acdf866102b..bf0cd6e7ca549 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.hudi +import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.common.config.TypedProperties +import org.apache.hudi.HoodieConversionUtils.toProperties +import org.apache.hudi.common.config.{DFSPropertiesConfiguration, TypedProperties} import org.apache.hudi.common.model.{OverwriteWithLatestAvroPayload, WriteOperationType} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME @@ -28,12 +30,13 @@ import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, MultiPartKeys import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.sql.InsertMode import org.apache.hudi.sync.common.HoodieSyncConfig -import org.apache.hudi.{DataSourceWriteOptions, HoodieWriterUtils} import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable import org.apache.spark.sql.hive.HiveExternalCatalog -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{isUsingHiveCatalog, withSparkConf} +import org.apache.spark.sql.hudi.HoodieOptionConfig.mapSqlOptionsToDataSourceWriteConfigs +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{isHoodieConfigKey, isUsingHiveCatalog} +import org.apache.spark.sql.hudi.ProvidesHoodieConfig.{combineOptions, withCombinedOptions} import org.apache.spark.sql.hudi.command.{SqlKeyGenerator, ValidateDuplicateKeyPayload} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -45,7 +48,6 @@ trait ProvidesHoodieConfig extends Logging { def buildHoodieConfig(hoodieCatalogTable: HoodieCatalogTable): Map[String, String] = { val sparkSession: SparkSession = hoodieCatalogTable.spark - val catalogProperties = hoodieCatalogTable.catalogProperties val tableConfig = hoodieCatalogTable.tableConfig // NOTE: Here we fallback to "" to make sure that null value is not overridden with @@ -56,11 +58,9 @@ trait ProvidesHoodieConfig extends Logging { require(hoodieCatalogTable.primaryKeys.nonEmpty, s"There are no primary key in table ${hoodieCatalogTable.table.identifier}, cannot execute update operator") - val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) - - withSparkConf(sparkSession, catalogProperties) { + withCombinedOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf) { Map.apply( "path" -> hoodieCatalogTable.tableLocation, RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), @@ -80,10 +80,8 @@ trait ProvidesHoodieConfig extends Logging { HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, - HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) - .filter { case(_, v) => v != null } } } @@ -107,12 +105,9 @@ trait ProvidesHoodieConfig extends Logging { val path = hoodieCatalogTable.tableLocation val tableType = hoodieCatalogTable.tableTypeName val tableConfig = hoodieCatalogTable.tableConfig - val catalogProperties = hoodieCatalogTable.catalogProperties - - val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf, extraOptions) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) - val parameters = withSparkConf(sparkSession, catalogProperties)() + val combinedOpts = combineOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf, extraOptions) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig, extraOptions) val partitionFieldsStr = hoodieCatalogTable.partitionFields.mkString(",") @@ -126,18 +121,21 @@ trait ProvidesHoodieConfig extends Logging { val keyGeneratorClassName = Option(tableConfig.getKeyGeneratorClassName) .getOrElse(classOf[ComplexKeyGenerator].getCanonicalName) - val enableBulkInsert = parameters.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key, - DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean || - parameters.get(DataSourceWriteOptions.OPERATION.key).exists(_.equalsIgnoreCase(WriteOperationType.BULK_INSERT.value)) + val enableBulkInsert = combinedOpts.getOrElse(DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key, + DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.defaultValue()).toBoolean val dropDuplicate = sparkSession.conf .getOption(INSERT_DROP_DUPS.key).getOrElse(INSERT_DROP_DUPS.defaultValue).toBoolean - val insertMode = InsertMode.of(parameters.getOrElse(DataSourceWriteOptions.SQL_INSERT_MODE.key, + val insertMode = InsertMode.of(combinedOpts.getOrElse(DataSourceWriteOptions.SQL_INSERT_MODE.key, DataSourceWriteOptions.SQL_INSERT_MODE.defaultValue())) val isNonStrictMode = insertMode == InsertMode.NON_STRICT val isPartitionedTable = hoodieCatalogTable.partitionFields.nonEmpty val hasPrecombineColumn = hoodieCatalogTable.preCombineKey.nonEmpty - val operation = + + // NOTE: Target operation could be overridden by the user, therefore if it has been provided as an input + // we'd prefer that value over auto-deduced operation. Otherwise, we deduce target operation type + val operationOverride = combinedOpts.get(DataSourceWriteOptions.OPERATION.key) + val operation = operationOverride.getOrElse { (enableBulkInsert, isOverwrite, dropDuplicate, isNonStrictMode, isPartitionedTable) match { case (true, _, _, false, _) => throw new IllegalArgumentException(s"Table with primaryKey can not use bulk insert in ${insertMode.value()} mode.") @@ -159,6 +157,7 @@ trait ProvidesHoodieConfig extends Logging { // for the rest case, use the insert operation case _ => INSERT_OPERATION_OPT_VAL } + } val payloadClassName = if (operation == UPSERT_OPERATION_OPT_VAL && tableType == COW_TABLE_TYPE_OPT_VAL && insertMode == InsertMode.STRICT) { @@ -173,10 +172,7 @@ trait ProvidesHoodieConfig extends Logging { classOf[OverwriteWithLatestAvroPayload].getCanonicalName } - - logInfo(s"Insert statement use write operation type: $operation, payloadClass: $payloadClassName") - - withSparkConf(sparkSession, catalogProperties) { + withCombinedOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf) { Map( "path" -> path, TABLE_TYPE.key -> tableType, @@ -190,7 +186,6 @@ trait ProvidesHoodieConfig extends Logging { PRECOMBINE_FIELD.key -> preCombineField, PARTITIONPATH_FIELD.key -> partitionFieldsStr, PAYLOAD_CLASS_NAME.key -> payloadClassName, - ENABLE_ROW_WRITER.key -> enableBulkInsert.toString, HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFieldsStr, HoodieSyncConfig.META_SYNC_ENABLED.key -> hiveSyncConfig.getString(HoodieSyncConfig.META_SYNC_ENABLED.key), @@ -200,26 +195,20 @@ trait ProvidesHoodieConfig extends Logging { HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME), HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), - HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"), - HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) - .filter { case (_, v) => v != null } } } - def buildHoodieDropPartitionsConfig( - sparkSession: SparkSession, - hoodieCatalogTable: HoodieCatalogTable, - partitionsToDrop: String): Map[String, String] = { + def buildHoodieDropPartitionsConfig(sparkSession: SparkSession, + hoodieCatalogTable: HoodieCatalogTable, + partitionsToDrop: String): Map[String, String] = { val partitionFields = hoodieCatalogTable.partitionFields.mkString(",") - val catalogProperties = hoodieCatalogTable.catalogProperties val tableConfig = hoodieCatalogTable.tableConfig - val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) - withSparkConf(sparkSession, catalogProperties) { + withCombinedOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf) { Map( "path" -> hoodieCatalogTable.tableLocation, TBL_NAME.key -> hoodieCatalogTable.tableName, @@ -238,14 +227,12 @@ trait ProvidesHoodieConfig extends Logging { HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields, HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) ) - .filter { case (_, v) => v != null } } } def buildHoodieDeleteTableConfig(hoodieCatalogTable: HoodieCatalogTable, sparkSession: SparkSession): Map[String, String] = { val path = hoodieCatalogTable.tableLocation - val catalogProperties = hoodieCatalogTable.catalogProperties val tableConfig = hoodieCatalogTable.tableConfig val tableSchema = hoodieCatalogTable.tableSchema val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase(Locale.ROOT)) @@ -254,14 +241,9 @@ trait ProvidesHoodieConfig extends Logging { assert(hoodieCatalogTable.primaryKeys.nonEmpty, s"There are no primary key defined in table ${hoodieCatalogTable.table.identifier}, cannot execute delete operation") - val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) - val options = hoodieCatalogTable.catalogProperties - val enableHive = isUsingHiveCatalog(sparkSession) - val partitionFields = hoodieCatalogTable.partitionFields.mkString(",") - - withSparkConf(sparkSession, options) { + withCombinedOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf) { Map( "path" -> path, RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), @@ -278,24 +260,20 @@ trait ProvidesHoodieConfig extends Logging { HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME), HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME), HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, - HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields, + HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> hoodieCatalogTable.partitionFields.mkString(","), HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), - HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) } } - def getHoodieProps(catalogProperties: Map[String, String], tableConfig: HoodieTableConfig, conf: SQLConf, extraOptions: Map[String, String] = Map.empty): TypedProperties = { - val options: Map[String, String] = catalogProperties ++ tableConfig.getProps.asScala.toMap ++ conf.getAllConfs ++ extraOptions - val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(options) - hoodieConfig.getProps - } + def buildHiveSyncConfig(sparkSession: SparkSession, + hoodieCatalogTable: HoodieCatalogTable, + tableConfig: HoodieTableConfig, + extraOptions: Map[String, String] = Map.empty): HiveSyncConfig = { + val combinedOpts = combineOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf, extraOptions) + val props = new TypedProperties(toProperties(combinedOpts)) - def buildHiveSyncConfig( - props: TypedProperties, - hoodieCatalogTable: HoodieCatalogTable, - sparkSession: SparkSession = SparkSession.active): HiveSyncConfig = { // Enable the hive sync by default if spark have enable the hive metastore. val enableHive = isUsingHiveCatalog(sparkSession) val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig(props) @@ -321,3 +299,39 @@ trait ProvidesHoodieConfig extends Logging { hiveSyncConfig } } + +object ProvidesHoodieConfig { + + def filterNullValues(opts: Map[String, String]): Map[String, String] = + opts.filter { case (_, v) => v != null } + + def withCombinedOptions(catalogTable: HoodieCatalogTable, + tableConfig: HoodieTableConfig, + sqlConf: SQLConf)(optionOverrides: Map[String, String] = Map.empty): Map[String, String] = { + combineOptions(catalogTable, tableConfig, sqlConf, optionOverrides) + } + + private def combineOptions(catalogTable: HoodieCatalogTable, + tableConfig: HoodieTableConfig, + sqlConf: SQLConf, + optionOverrides: Map[String, String] = Map.empty): Map[String, String] = { + // NOTE: Properties are merged in the following order of priority (first has the highest priority, last has the + // lowest, which is inverse to the ordering in the code): + // 1. (Extra) Option overrides + // 2. Spark SQL configs + // 3. Persisted Hudi's Table configs + // 4. Table's properties in Spark Catalog + // 5. Global DFS properties + DFSPropertiesConfiguration.getGlobalProps.asScala.toMap ++ + // NOTE: Catalog table provided t/h `TBLPROPERTIES` clause might contain Spark SQL specific + // properties that need to be mapped into Hudi's conventional ones + mapSqlOptionsToDataSourceWriteConfigs(catalogTable.catalogProperties) ++ + tableConfig.getProps.asScala.toMap ++ + filterHoodieConfigs(sqlConf.getAllConfs) ++ + filterNullValues(optionOverrides) + } + + private def filterHoodieConfigs(opts: Map[String, String]): Map[String, String] = + opts.filterKeys(isHoodieConfigKey) + +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala deleted file mode 100644 index 631644121c133..0000000000000 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/SerDeUtils.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hudi - -import org.apache.hudi.common.util.BinaryUtil -import org.apache.spark.SparkConf -import org.apache.spark.serializer.{KryoSerializer, SerializerInstance} - -import java.nio.ByteBuffer - - -object SerDeUtils { - - private val SERIALIZER_THREAD_LOCAL = new ThreadLocal[SerializerInstance] { - - override protected def initialValue: SerializerInstance = { - new KryoSerializer(new SparkConf(true)).newInstance() - } - } - - def toBytes(o: Any): Array[Byte] = { - val buf = SERIALIZER_THREAD_LOCAL.get.serialize(o) - BinaryUtil.toBytes(buf) - } - - def toObject(bytes: Array[Byte]): Any = { - SERIALIZER_THREAD_LOCAL.get.deserialize(ByteBuffer.wrap(bytes)) - } -} 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 820891ce956e4..169e8899926ba 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 @@ -590,7 +590,8 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic CreateHoodieTableCommand(table, ignoreIfExists) // Rewrite the DropTableCommand to DropHoodieTableCommand case DropTableCommand(tableName, ifExists, false, purge) - if sparkAdapter.isHoodieTable(tableName, sparkSession) => + if sparkSession.sessionState.catalog.tableExists(tableName) + && sparkAdapter.isHoodieTable(tableName, sparkSession) => DropHoodieTableCommand(tableName, ifExists, false, purge) // Rewrite the AlterTableDropPartitionCommand to AlterHoodieTableDropPartitionCommand case AlterTableDropPartitionCommand(tableName, specs, ifExists, purge, retainData) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 1f8d009530146..0e19514e28c7b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.DataSourceWriteOptions import org.apache.hudi.common.util.ValidationUtils.checkState +import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hive.HiveSyncConfigHolder import org.apache.hudi.sql.InsertMode import org.apache.hudi.sync.common.util.ConfigUtils @@ -92,13 +93,11 @@ case class CreateHoodieTableAsSelectCommand( hoodieCatalogTable.initHoodieTable() val tableProperties = hoodieCatalogTable.catalogProperties - // NOTE: Users might be specifying write-configuration (inadvertently) as options or table properties - // in CTAS, therefore we need to make sure that these are appropriately propagated to the - // write operation - val options = tableProperties ++ Map( + val options = Map( HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString, HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tableProperties.asJava), HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(updatedTable.properties.asJava), + HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> "false", DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index b148d3d510fb1..3994d3cdf12ff 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -22,6 +22,7 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME +import org.apache.hudi.config.HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE import org.apache.hudi.exception.HoodieException import org.apache.hudi.hive.HiveSyncConfigHolder import org.apache.hudi.sync.common.HoodieSyncConfig @@ -35,10 +36,11 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeRef import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._ import org.apache.spark.sql.hudi.HoodieSqlUtils.getMergeIntoTargetTableId +import org.apache.spark.sql.hudi.ProvidesHoodieConfig.withCombinedOptions import org.apache.spark.sql.hudi.command.MergeIntoHoodieTableCommand.CoercedAttributeReference import org.apache.spark.sql.hudi.command.payload.ExpressionPayload import org.apache.spark.sql.hudi.command.payload.ExpressionPayload._ -import org.apache.spark.sql.hudi.{ProvidesHoodieConfig, SerDeUtils} +import org.apache.spark.sql.hudi.ProvidesHoodieConfig import org.apache.spark.sql.types.{BooleanType, StructType} import java.util.Base64 @@ -323,7 +325,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie }).toMap // Serialize the Map[UpdateCondition, UpdateAssignments] to base64 string val serializedUpdateConditionAndExpressions = Base64.getEncoder - .encodeToString(SerDeUtils.toBytes(updateConditionToAssignments)) + .encodeToString(Serializer.toBytes(updateConditionToAssignments)) writeParams += (PAYLOAD_UPDATE_CONDITION_AND_ASSIGNMENTS -> serializedUpdateConditionAndExpressions) @@ -333,7 +335,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie .getOrElse(Literal.create(true, BooleanType)) // Serialize the Map[DeleteCondition, empty] to base64 string val serializedDeleteCondition = Base64.getEncoder - .encodeToString(SerDeUtils.toBytes(Map(deleteCondition -> Seq.empty[Assignment]))) + .encodeToString(Serializer.toBytes(Map(deleteCondition -> Seq.empty[Assignment]))) writeParams += (PAYLOAD_DELETE_CONDITION -> serializedDeleteCondition) } @@ -404,7 +406,7 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie rewriteCondition -> formatAssignments }).toMap Base64.getEncoder.encodeToString( - SerDeUtils.toBytes(insertConditionAndAssignments)) + Serializer.toBytes(insertConditionAndAssignments)) } /** @@ -482,8 +484,6 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie val targetTableDb = targetTableIdentify.database.getOrElse("default") val targetTableName = targetTableIdentify.identifier val path = hoodieCatalogTable.tableLocation - // force to use ExpressionPayload as WRITE_PAYLOAD_CLASS_NAME in MergeIntoHoodieTableCommand - val catalogProperties = hoodieCatalogTable.catalogProperties + (PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName) val tableConfig = hoodieCatalogTable.tableConfig val tableSchema = hoodieCatalogTable.tableSchema val partitionColumns = tableConfig.getPartitionFieldProp.split(",").map(_.toLowerCase) @@ -494,10 +494,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie // TODO(HUDI-3456) clean up val preCombineField = hoodieCatalogTable.preCombineKey.getOrElse("") - val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) - withSparkConf(sparkSession, catalogProperties) { + withCombinedOptions(hoodieCatalogTable, tableConfig, sparkSession.sqlContext.conf) { Map( "path" -> path, RECORDKEY_FIELD.key -> tableConfig.getRecordKeyFieldProp, @@ -516,12 +515,18 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), - HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"), // set the default parallelism to 200 for sql - HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"), - HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"), - SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL + SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL, + PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName, + + // NOTE: We have to explicitly override following configs to make sure no schema validation is performed + // as schema of the incoming dataset might be diverging from the table's schema (full schemas' + // compatibility b/w table's schema and incoming one is not necessary in this case since we can + // be cherry-picking only selected columns from the incoming dataset to be inserted/updated in the + // target table, ie partially updating) + AVRO_SCHEMA_VALIDATE_ENABLE.key -> "false", + RECONCILE_SCHEMA.key -> "false", + "hoodie.datasource.write.schema.canonicalize" -> "false" ) - .filter { case (_, v) => v != null } } } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala index 55932237d95f5..d7f33fd569cf2 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/payload/ExpressionPayload.scala @@ -25,16 +25,18 @@ import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord} -import org.apache.hudi.common.util.{ValidationUtils, Option => HOption} +import org.apache.hudi.common.util.{BinaryUtil, ValidationUtils, Option => HOption} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.io.HoodieWriteHandle import org.apache.hudi.sql.IExpressionEvaluator +import org.apache.spark.serializer.{KryoSerializer, SerializerInstance} +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.sql.avro.{AvroSerializer, SchemaConverters} import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.hudi.SerDeUtils import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.{getEvaluator, getMergedSchema, setWriteSchema} import org.apache.spark.sql.types.{StructField, StructType} +import java.nio.ByteBuffer import java.util.{Base64, Properties} import java.util.function.Function import scala.collection.JavaConverters._ @@ -296,7 +298,7 @@ object ExpressionPayload { new Function[String, Map[IExpressionEvaluator, IExpressionEvaluator]] { override def apply(t: String): Map[IExpressionEvaluator, IExpressionEvaluator] = { val serializedBytes = Base64.getDecoder.decode(t) - val conditionAssignments = SerDeUtils.toObject(serializedBytes) + val conditionAssignments = Serializer.toObject(serializedBytes) .asInstanceOf[Map[Expression, Seq[Expression]]] // Do the CodeGen for condition expression and assignment expression conditionAssignments.map { @@ -339,5 +341,49 @@ object ExpressionPayload { } case class TupleSchema(first: Schema, second: Schema) + + /** + * This object differs from Hudi's generic [[SerializationUtils]] in its ability to serialize + * Spark's internal structures (various [[Expression]]s) + * + * For that purpose we re-use Spark's [[KryoSerializer]] instance sharing configuration + * with enclosing [[SparkEnv]]. This is necessary to make sure that this particular instance of Kryo + * user for serialization of Spark's internal structures (like [[Expression]]s) is configured + * appropriately (class-loading, custom serializers, etc) + * + * TODO rebase on Spark's SerializerSupport + */ + private[hudi] object Serializer { + + // NOTE: This is only Spark >= 3.0 + private val KRYO_USE_POOL_CONFIG_KEY = "spark.kryo.pool" + + private lazy val conf = { + val conf = Option(SparkEnv.get) + // To make sure we're not modifying existing environment's [[SparkConf]] + // we're cloning it here + .map(_.conf.clone) + .getOrElse(new SparkConf) + // This serializer is configured as thread-local, hence there's no need for + // pooling + conf.set(KRYO_USE_POOL_CONFIG_KEY, "false") + conf + } + + private val SERIALIZER_THREAD_LOCAL = new ThreadLocal[SerializerInstance] { + override protected def initialValue: SerializerInstance = { + new KryoSerializer(conf).newInstance() + } + } + + def toBytes(o: Any): Array[Byte] = { + val buf = SERIALIZER_THREAD_LOCAL.get.serialize(o) + BinaryUtil.toBytes(buf) + } + + def toObject(bytes: Array[Byte]): Any = { + SERIALIZER_THREAD_LOCAL.get.deserialize(ByteBuffer.wrap(bytes)) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HiveSyncProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HiveSyncProcedure.scala index 24944b1270404..55065c686cf7a 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HiveSyncProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HiveSyncProcedure.scala @@ -81,8 +81,7 @@ class HiveSyncProcedure extends BaseProcedure with ProcedureBuilder hiveConf.addResource(hadoopConf) val tableConfig = hoodieCatalogTable.tableConfig - val hoodieProps = getHoodieProps(hoodieCatalogTable.catalogProperties, tableConfig, sqlConf) - val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable) + val hiveSyncConfig = buildHiveSyncConfig(sparkSession, hoodieCatalogTable, tableConfig) var hiveSyncTool: HiveSyncTool = null try { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index 11f0fc97856e9..b98d6eb5af19e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -21,16 +21,19 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SerializationUtils; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; +import org.apache.hudi.metadata.HoodieMetadataPayload; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.avro.Conversions; @@ -39,6 +42,7 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -59,6 +63,7 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import java.io.IOException; import java.math.BigDecimal; import java.time.LocalDate; import java.util.ArrayList; @@ -323,4 +328,28 @@ private static Stream testAutoModifyParquetWriteLegacyFormatParameter {false, true, true}, {false, false, false} }).map(Arguments::of); } + + @Test + public void testSerHoodieMetadataPayload() throws IOException { + String partitionPath = "2022/10/01"; + String fileName = "file.parquet"; + String targetColName = "c1"; + + HoodieColumnRangeMetadata columnStatsRecord = + HoodieColumnRangeMetadata.create(fileName, targetColName, 0, 500, 0, 100, 12345, 12345); + + HoodieRecord hoodieMetadataPayload = + HoodieMetadataPayload.createColumnStatsRecords(partitionPath, Collections.singletonList(columnStatsRecord), false) + .findFirst().get(); + + IndexedRecord record = hoodieMetadataPayload.getData().getInsertValue(null).get(); + byte[] recordToBytes = HoodieAvroUtils.indexedRecordToBytes(record); + GenericRecord genericRecord = HoodieAvroUtils.bytesToAvro(recordToBytes, record.getSchema()); + + HoodieMetadataPayload genericRecordHoodieMetadataPayload = new HoodieMetadataPayload(Option.of(genericRecord)); + byte[] bytes = SerializationUtils.serialize(genericRecordHoodieMetadataPayload); + HoodieMetadataPayload deserGenericRecordHoodieMetadataPayload = SerializationUtils.deserialize(bytes); + + assertEquals(genericRecordHoodieMetadataPayload, deserGenericRecordHoodieMetadataPayload); + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala new file mode 100644 index 0000000000000..ad476fb38f38e --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroSchemaResolutionSupport.scala @@ -0,0 +1,794 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ +package org.apache.hudi + +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.testutils.HoodieClientTestBase +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.junit.jupiter.api.{AfterEach, BeforeEach} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import scala.language.postfixOps + +/** + * Test cases to validate Hudi's support for writing and reading when evolving schema implicitly via Avro's Schema Resolution + * Note: Test will explicitly write into different partitions to ensure that a Hudi table will have multiple filegroups with different schemas. + */ +class TestAvroSchemaResolutionSupport extends HoodieClientTestBase { + + var spark: SparkSession = _ + val commonOpts: Map[String, String] = Map( + HoodieWriteConfig.TBL_NAME.key -> "hoodie_avro_schema_resolution_support", + "hoodie.insert.shuffle.parallelism" -> "1", + "hoodie.upsert.shuffle.parallelism" -> "1", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "id", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "name", + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.SimpleKeyGenerator", + HoodieMetadataConfig.ENABLE.key -> "false" + ) + + /** + * Setup method running before each test. + */ + @BeforeEach override def setUp(): Unit = { + setTableName("hoodie_avro_schema_resolution_support") + initPath() + initSparkContexts() + spark = sqlContext.sparkSession + } + + @AfterEach override def tearDown(): Unit = { + cleanupSparkContexts() + } + + def castColToX(x: Int, colToCast: String, df: DataFrame): DataFrame = x match { + case 0 => df.withColumn(colToCast, df.col(colToCast).cast("long")) + case 1 => df.withColumn(colToCast, df.col(colToCast).cast("float")) + case 2 => df.withColumn(colToCast, df.col(colToCast).cast("double")) + case 3 => df.withColumn(colToCast, df.col(colToCast).cast("binary")) + case 4 => df.withColumn(colToCast, df.col(colToCast).cast("string")) + } + + def initialiseTable(df: DataFrame, saveDir: String, isCow: Boolean = true): Unit = { + val opts = if (isCow) { + commonOpts ++ Map(DataSourceWriteOptions.TABLE_TYPE.key -> "COPY_ON_WRITE") + } else { + commonOpts ++ Map(DataSourceWriteOptions.TABLE_TYPE.key -> "MERGE_ON_READ") + } + + df.write.format("hudi") + .options(opts) + .mode("overwrite") + .save(saveDir) + } + + def upsertData(df: DataFrame, saveDir: String, isCow: Boolean = true): Unit = { + val opts = if (isCow) { + commonOpts ++ Map(DataSourceWriteOptions.TABLE_TYPE.key -> "COPY_ON_WRITE") + } else { + commonOpts ++ Map(DataSourceWriteOptions.TABLE_TYPE.key -> "MERGE_ON_READ") + } + + df.write.format("hudi") + .options(opts) + .mode("append") + .save(saveDir) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testDataTypePromotions(isCow: Boolean): Unit = { + // test to read tables with columns that are promoted via avro schema resolution + val tempRecordPath = basePath + "/record_tbl/" + val _spark = spark + import _spark.implicits._ + + val colToCast = "userId" + val df1 = Seq((1, 100, "aaa")).toDF("id", "userid", "name") + val df2 = Seq((2, 200L, "bbb")).toDF("id", "userid", "name") + + def prepDataFrame(df: DataFrame, colInitType: String): DataFrame = { + // convert int to string first before conversion to binary + // after which, initialise df with initType + if (colInitType == "binary") { + val castDf = df.withColumn(colToCast, df.col(colToCast).cast("string")) + castDf.withColumn(colToCast, castDf.col(colToCast).cast(colInitType)) + } else { + df.withColumn(colToCast, df.col(colToCast).cast(colInitType)) + } + } + + def doTest(colInitType: String, start: Int, end: Int): Unit = { + for (a <- Range(start, end)) { + try { + Console.println(s"Performing test: $a with initialColType of: $colInitType") + + // convert int to string first before conversion to binary + val initDF = prepDataFrame(df1, colInitType) + initDF.printSchema() + initDF.show(false) + + // recreate table + initialiseTable(initDF, tempRecordPath, isCow) + + // perform avro supported casting + var upsertDf = prepDataFrame(df2, colInitType) + upsertDf = castColToX(a, colToCast, upsertDf) + upsertDf.printSchema() + upsertDf.show(false) + + // upsert + upsertData(upsertDf, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + + assert(true) + } catch { + case e: Exception => { + // e.printStackTrace() + // Console.println(s"Test $a failed with error: ${e.getMessage}") + assert(false, e) + } + } + } + } + + // INT -> [Long, Float, Double, String] + doTest("int", 0, 3) + // Long -> [Float, Double, String] + doTest("long", 1, 3) + // Float -> [Double, String] + doTest("float", 2, 3) + // String -> [Bytes] + doTest("string", 3, 4) + // Bytes -> [String] + doTest("binary", 4, 5) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testAddNewColumn(isCow: Boolean): Unit = { + // test to add a column + val tempRecordPath = basePath + "/record_tbl/" + val _spark = spark + import _spark.implicits._ + + val df1 = Seq((1, 100, "aaa")).toDF("id", "userid", "name") + val df2 = Seq((2, 200, "newCol", "bbb")).toDF("id", "userid", "newcol", "name") + + // convert int to string first before conversion to binary + val initDF = df1 + initDF.printSchema() + initDF.show(false) + + // recreate table + initialiseTable(initDF, tempRecordPath, isCow) + + // perform avro supported operation of adding a new column at the end of the table + val upsertDf = df2 + upsertDf.printSchema() + upsertDf.show(false) + + // upsert + upsertData(upsertDf, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testDeleteColumn(isCow: Boolean): Unit = { + // test to delete a column + val tempRecordPath = basePath + "/record_tbl/" + val _spark = spark + import _spark.implicits._ + + val df1 = Seq((1, 100, "aaa")).toDF("id", "userid", "name") + val df2 = Seq((2, "bbb")).toDF("id", "name") + + // convert int to string first before conversion to binary + val initDF = df1 + initDF.printSchema() + initDF.show(false) + + // recreate table + initialiseTable(initDF, tempRecordPath, isCow) + + // perform avro supported operation of deleting a column + val upsertDf = df2 + upsertDf.printSchema() + upsertDf.show(false) + + // upsert + upsertData(upsertDf, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testColumnPositionChange(isCow: Boolean): Unit = { + // test to change column positions + val tempRecordPath = basePath + "/record_tbl/" + val _spark = spark + import _spark.implicits._ + + val df1 = Seq((1, 100, "col1", "aaa")).toDF("id", "userid", "newcol", "name") + val df2 = Seq((2, "col2", 200, "bbb")).toDF("id", "newcol", "userid", "name") + + // convert int to string first before conversion to binary + val initDF = df1 + initDF.printSchema() + initDF.show(false) + + // recreate table + initialiseTable(initDF, tempRecordPath, isCow) + + // perform avro supported operation of deleting a column + val upsertDf = df2 + upsertDf.printSchema() + upsertDf.show(false) + + // upsert + upsertData(upsertDf, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfStructsAddNewColumn(isCow: Boolean): Unit = { + // test to add a field to a STRUCT in a column of ARRAY< STRUCT<..> > type + + // there is a bug on Spark3 that will prevent Array[Map/Struct] schema evolved tables form being read + // bug fix: https://github.com/apache/spark/commit/32a393395ee43b573ae75afba591b587ca51879b + // bug fix is only available Spark >= v3.1.3 + if (HoodieSparkUtils.isSpark2 || (HoodieSparkUtils.isSpark3 && HoodieSparkUtils.gteqSpark3_1_3)) { + val tempRecordPath = basePath + "/record_tbl/" + val arrayStructData = Seq( + Row(1, 100, List(Row("Java", "XX", 120), Row("Scala", "XA", 300)), "aaa") + ) + val arrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("name", StringType) + .add("author", StringType) + .add("pages", IntegerType))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayStructData), arrayStructSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // add a column to array of struct + val newArrayStructData = Seq( + Row(2, 200, List(Row("JavaV2", "XXX", 130, 20), Row("ScalaV2", "XXA", 310, 40)), "bbb") + ) + val newArrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("name", StringType) + .add("author", StringType) + .add("pages", IntegerType) + .add("progress", IntegerType) + )) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayStructData), newArrayStructSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfStructsChangeColumnType(isCow: Boolean): Unit = { + // test to change the type of a field from a STRUCT in a column of ARRAY< STRUCT<..> > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayStructData = Seq( + Row(1, 100, List(Row("Java", "XX", 120), Row("Scala", "XA", 300)), "aaa") + ) + val arrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("name", StringType) + .add("author", StringType) + .add("pages", IntegerType))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayStructData), arrayStructSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // add a column to array of struct + val newArrayStructData = Seq( + Row(2, 200, List(Row("XXX", "JavaV2", 130L), Row("XXA", "ScalaV2", 310L)), "bbb") + ) + val newArrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("author", StringType) + .add("name", StringType) + .add("pages", LongType))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayStructData), newArrayStructSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfStructsChangeColumnPosition(isCow: Boolean): Unit = { + // test to change the position of a field from a STRUCT in a column of ARRAY< STRUCT<..> > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayStructData = Seq( + Row(1, 100, List(Row("Java", "XX", 120), Row("Scala", "XA", 300)), "aaa") + ) + val arrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("name", StringType) + .add("author", StringType) + .add("pages", IntegerType))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayStructData), arrayStructSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // add a column to array of struct + val newArrayStructData = Seq( + Row(2, 200, List(Row(130, "JavaV2", "XXX"), Row(310, "ScalaV2", "XXA")), "bbb") + ) + val newArrayStructSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("language", ArrayType(new StructType() + .add("pages", IntegerType) + .add("name", StringType) + .add("author", StringType))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayStructData), newArrayStructSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfMapsChangeValueType(isCow: Boolean): Unit = { + // test to change the value type of a MAP in a column of ARRAY< MAP > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, List(Map("2022-12-01" -> 120), Map("2022-12-02" -> 130)), "aaa") + ) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("salesMap", ArrayType( + new MapType(StringType, IntegerType, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // change value type from integer to long + val newArrayMapData = Seq( + Row(2, 200, List(Map("2022-12-01" -> 220L), Map("2022-12-02" -> 230L)), "bbb") + ) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("salesMap", ArrayType( + new MapType(StringType, LongType, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfMapsStructChangeFieldType(isCow: Boolean): Unit = { + // test to change a field type of a STRUCT in a column of ARRAY< MAP< k,STRUCT<..> > > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, + List( + Map("2022-12-01" -> Row("a1", "b1", 20)), + Map("2022-12-02" -> Row("a2", "b2", 30)) + ), + "aaa") + ) + val innerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", IntegerType) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, innerStructSchema, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // change inner struct's type from integer to long + val newArrayMapData = Seq( + Row(2, 200, + List( + Map("2022-12-03" -> Row("a3", "b3", 40L)), + Map("2022-12-04" -> Row("a4", "b4", 50L)) + ), + "bbb") + ) + val newInnerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", LongType) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, newInnerStructSchema, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfMapsStructAddField(isCow: Boolean): Unit = { + // test to add a field to a STRUCT in a column of ARRAY< MAP< k,STRUCT<..> > > type + + // there is a bug on Spark3 that will prevent Array[Map/Struct] schema evolved tables form being read + // bug fix: https://github.com/apache/spark/commit/32a393395ee43b573ae75afba591b587ca51879b + // bug fix is only available Spark >= v3.1.3 + if (HoodieSparkUtils.isSpark2 || (HoodieSparkUtils.isSpark3 && HoodieSparkUtils.gteqSpark3_1_3)) { + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, + List( + Map("2022-12-01" -> Row("a1", "b1", 20)), + Map("2022-12-02" -> Row("a2", "b2", 30)) + ), + "aaa") + ) + val innerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", IntegerType) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, innerStructSchema, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // add a new column + val newArrayMapData = Seq( + Row(2, 200, + List( + Map("2022-12-01" -> Row("a3", "b3", 20, 40)), + Map("2022-12-02" -> Row("a4", "b4", 30, 40)) + ), + "bbb") + ) + val newInnerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", IntegerType) + .add("col4", IntegerType) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, newInnerStructSchema, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfMapsStructChangeFieldPosition(isCow: Boolean): Unit = { + // test to change the position of fields of a STRUCT in a column of ARRAY< MAP< k,STRUCT<..> > > type + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, + List( + Map("2022-12-01" -> Row("a1", "b1", 20)), + Map("2022-12-02" -> Row("a2", "b2", 30)) + ), + "aaa") + ) + val innerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", IntegerType) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, innerStructSchema, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // change column position + val newArrayMapData = Seq( + Row(2, 200, + List( + Map("2022-12-01" -> Row("a3", 40, "b3")), + Map("2022-12-02" -> Row("a4", 50, "b4")) + ), + "bbb") + ) + val newInnerStructSchema = new StructType() + .add("col1", StringType) + .add("col3", IntegerType) + .add("col2", StringType) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, newInnerStructSchema, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testArrayOfMapsStructDeleteField(isCow: Boolean): Unit = { + // test to delete a field of a STRUCT in a column of ARRAY< MAP< k,STRUCT<..> > > type + + val tempRecordPath = basePath + "/record_tbl/" + val arrayMapData = Seq( + Row(1, 100, + List( + Map("2022-12-01" -> Row("a1", "b1", 20)), + Map("2022-12-02" -> Row("a2", "b2", 30)) + ), + "aaa") + ) + val innerStructSchema = new StructType() + .add("col1", StringType) + .add("col2", StringType) + .add("col3", IntegerType) + val arrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, innerStructSchema, true))) + .add("name", StringType) + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(arrayMapData), arrayMapSchema) + df1.printSchema() + df1.show(false) + + // recreate table + initialiseTable(df1, tempRecordPath, isCow) + + // change column position + val newArrayMapData = Seq( + Row(2, 200, + List( + Map("2022-12-01" -> Row("a3", 40)), + Map("2022-12-02" -> Row("a4", 50)) + ), + "bbb") + ) + val newInnerStructSchema = new StructType() + .add("col1", StringType) + .add("col3", IntegerType) + val newArrayMapSchema = new StructType() + .add("id", IntegerType) + .add("userid", IntegerType) + .add("structcol", ArrayType( + new MapType(StringType, newInnerStructSchema, true))) + .add("name", StringType) + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(newArrayMapData), newArrayMapSchema) + df2.printSchema() + df2.show(false) + // upsert + upsertData(df2, tempRecordPath, isCow) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } + + @ParameterizedTest + @ValueSource(booleans = Array(true, false)) + def testComplexOperationsOnTable(isCow: Boolean): Unit = { + // test a series of changes on a Hudi table + + var defaultPartitionIdx = 0 + + def newPartition: String = { + defaultPartitionIdx = defaultPartitionIdx + 1 + "aaa" + defaultPartitionIdx + } + + val tempRecordPath = basePath + "/record_tbl/" + val _spark = spark + import _spark.implicits._ + + // 1. Initialise table + val df1 = Seq((1, 100, newPartition)).toDF("id", "userid", "name") + df1.printSchema() + df1.show(false) + initialiseTable(df1, tempRecordPath, isCow) + + // 2. Promote INT type to LONG into a different partition + val df2 = Seq((2, 200L, newPartition)).toDF("id", "userid", "name") + df2.printSchema() + df2.show(false) + upsertData(df2, tempRecordPath, isCow) + + // 3. Promote LONG to FLOAT + var df3 = Seq((3, 300, newPartition)).toDF("id", "userid", "name") + df3 = df3.withColumn("userid", df3.col("userid").cast("float")) + df3.printSchema() + df3.show(false) + upsertData(df3, tempRecordPath) + + // 4. Promote FLOAT to DOUBLE + var df4 = Seq((4, 400, newPartition)).toDF("id", "userid", "name") + df4 = df4.withColumn("userid", df4.col("userid").cast("float")) + df4.printSchema() + df4.show(false) + upsertData(df4, tempRecordPath) + + // 5. Add two new column + var df5 = Seq((5, 500, "newcol1", "newcol2", newPartition)).toDF("id", "userid", "newcol1", "newcol2", "name") + df5 = df5.withColumn("userid", df5.col("userid").cast("float")) + df5.printSchema() + df5.show(false) + upsertData(df5, tempRecordPath) + + // 6. Delete a column + var df6 = Seq((6, 600, "newcol1", newPartition)).toDF("id", "userid", "newcol1", "name") + df6 = df6.withColumn("userid", df6.col("userid").cast("float")) + df6.printSchema() + df6.show(false) + upsertData(df6, tempRecordPath) + + // 7. Rearrange column position + var df7 = Seq((7, "newcol1", 700, newPartition)).toDF("id", "newcol1", "userid", "name") + df7 = df7.withColumn("userid", df7.col("userid").cast("float")) + df7.printSchema() + df7.show(false) + upsertData(df7, tempRecordPath) + + // read out the table + val readDf = spark.read.format("hudi").load(tempRecordPath) + readDf.printSchema() + readDf.show(false) + readDf.foreach(_ => {}) + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala index 15b6751328cba..a54116f1e790f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSourceStorage.scala @@ -29,7 +29,9 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.TimestampBasedKeyGenerator import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness.getSparkSqlConf import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} +import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} @@ -58,6 +60,8 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness { val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" + override def conf: SparkConf = conf(getSparkSqlConf) + @ParameterizedTest @CsvSource(value = Array( "true|org.apache.hudi.keygen.SimpleKeyGenerator|_row_key", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala index 8cf6b4174c9f2..107514c1de830 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala @@ -26,8 +26,10 @@ import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.common.util.StringUtils import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness.getSparkSqlConf import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.log4j.LogManager +import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} @@ -57,6 +59,8 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness { val verificationCol: String = "driver" val updatedVerificationVal: String = "driver_update" + override def conf: SparkConf = conf(getSparkSqlConf) + @ParameterizedTest @CsvSource(Array( "true,", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala index 57ebd038f2ae1..e3832eb7399a8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithBucket.scala @@ -150,4 +150,33 @@ class TestDataSourceForBucketIndex extends HoodieClientTestBase { assertEquals(100, hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count()) } + + @Test def testInsertOverwrite(): Unit = { + val partitionPaths = new Array[String](1) + partitionPaths.update(0, "2020/01/10") + val newDataGen = new HoodieTestDataGenerator(partitionPaths) + val records1 = recordsToStrings(newDataGen.generateInserts("001", 100)).toList + val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2)) + inputDF1.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000")) + val records2 = recordsToStrings(newDataGen.generateInserts("002", 20)).toList + val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2)) + inputDF2.write.format("org.apache.hudi") + .options(commonOpts) + .option("hoodie.compact.inline", "false") + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL) + .option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val hudiSnapshotDF1 = spark.read.format("org.apache.hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) + .load(basePath + "/*/*/*/*") + assertEquals(20, hudiSnapshotDF1.count()) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala index 02e0ee6dfd9bc..995fa6adb6a74 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala @@ -24,9 +24,13 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness.getSparkSqlConf +import org.apache.spark.SparkConf import org.apache.spark.sql.SaveMode import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.{Tag, Test} +import org.junit.jupiter.api.Tag +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource import scala.collection.JavaConverters._ @@ -45,8 +49,11 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" ) - @Test - def testReadability(): Unit = { + override def conf: SparkConf = conf(getSparkSqlConf) + + @ParameterizedTest + @ValueSource(ints = Array(1, 5)) + def testReadability(compactNumDeltaCommits: Int): Unit = { val dataGen = new HoodieTestDataGenerator() val metadataOpts: Map[String, String] = Map( @@ -55,7 +62,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn ) val combinedOpts: Map[String, String] = commonOpts ++ metadataOpts ++ - Map(HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1") + Map(HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> compactNumDeltaCommits.toString) // Insert records val newRecords = dataGen.generateInserts("001", 100) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala index 00ab7091445db..b50aebd1f5d70 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -18,6 +18,7 @@ package org.apache.hudi.functional import org.apache.avro.Schema +import org.apache.calcite.runtime.SqlFunctions.abs import org.apache.hudi.HoodieBaseRelation.projectSchema import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.model.{HoodieRecord, OverwriteNonDefaultsWithLatestAvroPayload} @@ -26,12 +27,14 @@ import org.apache.hudi.common.testutils.{HadoopMapRedUtils, HoodieTestDataGenera import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} import org.apache.hudi.keygen.NonpartitionedKeyGenerator import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness.getSparkSqlConf import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD} import org.apache.parquet.hadoop.util.counters.BenchmarkCounter +import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.{Dataset, HoodieUnsafeUtils, Row, SaveMode} -import org.junit.jupiter.api.Assertions.{assertEquals, fail} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.{Disabled, Tag, Test} import scala.collection.JavaConverters._ @@ -54,6 +57,8 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getName ) + override def conf: SparkConf = conf(getSparkSqlConf) + @Disabled("Currently disabled b/c of the fallback to HadoopFsRelation") @Test def testBaseFileOnlyViewRelation(): Unit = { @@ -236,9 +241,9 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with else if (HoodieSparkUtils.isSpark2) // TODO re-enable tests (these tests are very unstable currently) Array( - ("rider", -1), - ("rider,driver", -1), - ("rider,driver,tip_history", -1)) + ("rider", 14160), + ("rider,driver", 14160), + ("rider,driver,tip_history", 14160)) else fail("Only Spark 3 and Spark 2 are currently supported") @@ -327,11 +332,8 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with else targetRecordCount assertEquals(expectedRecordCount, rows.length) - if (expectedBytesRead != -1) { - assertEquals(expectedBytesRead, bytesRead) - } else { - logWarning(s"Not matching bytes read ($bytesRead)") - } + // verify within 10% of margin. + assertTrue((abs(expectedBytesRead - bytesRead) / expectedBytesRead) < 0.1) val readColumns = targetColumns ++ relation.mandatoryFields val (_, projectedStructType, _) = projectSchema(Left(tableState.schema), readColumns) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 1382bafb7621a..dbcc4789564b3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -30,7 +30,7 @@ import org.apache.hudi.testutils.HoodieClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} import org.apache.log4j.LogManager import org.apache.spark.sql._ -import org.apache.spark.sql.streaming.{OutputMode, Trigger} +import org.apache.spark.sql.streaming.{DataStreamWriter, OutputMode, StreamingQuery, Trigger} import org.apache.spark.sql.types.StructType import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach} @@ -47,7 +47,9 @@ import scala.concurrent.{Await, Future} */ class TestStructuredStreaming extends HoodieClientTestBase { private val log = LogManager.getLogger(getClass) - var spark: SparkSession = null + + var spark: SparkSession = _ + val commonOpts = Map( "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4", @@ -58,41 +60,29 @@ class TestStructuredStreaming extends HoodieClientTestBase { ) @BeforeEach override def setUp() { - initPath() - initSparkContexts() + super.setUp() spark = sqlContext.sparkSession - initTestDataGenerator() - initFileSystem() - initTimelineService() - } - - @AfterEach override def tearDown() = { - cleanupTimelineService() - cleanupSparkContexts() - cleanupTestDataGenerator() - cleanupFileSystem() + // We set stop to timeout after 30s to avoid blocking things indefinitely + spark.conf.set("spark.sql.streaming.stopTimeout", 30000) } - def initStreamingWriteFuture(schema: StructType, sourcePath: String, destPath: String, hudiOptions: Map[String, String]): Future[Unit] = { - // define the source of streaming + def initWritingStreamingQuery(schema: StructType, + sourcePath: String, + destPath: String, + hudiOptions: Map[String, String]): StreamingQuery = { val streamingInput = spark.readStream .schema(schema) .json(sourcePath) - Future { - println("streaming starting") - //'writeStream' can be called only on streaming Dataset/DataFrame - streamingInput - .writeStream - .format("org.apache.hudi") - .options(hudiOptions) - .trigger(Trigger.ProcessingTime(100)) - .option("checkpointLocation", basePath + "/checkpoint") - .outputMode(OutputMode.Append) - .start(destPath) - .awaitTermination(10000) - println("streaming ends") - } + + streamingInput + .writeStream + .format("org.apache.hudi") + .options(hudiOptions) + .trigger(Trigger.ProcessingTime(1000)) + .option("checkpointLocation", basePath + "/checkpoint") + .outputMode(OutputMode.Append) + .start(destPath) } def initStreamingSourceAndDestPath(sourceDirName: String, destDirName: String): (String, String) = { @@ -142,11 +132,13 @@ class TestStructuredStreaming extends HoodieClientTestBase { } else { getOptsWithTableType(tableType) } - val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, hudiOptions) + + val streamingQuery = initWritingStreamingQuery(inputDF1.schema, sourcePath, destPath, hudiOptions) val f2 = Future { inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) // wait for spark streaming to process one microbatch + val currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000")) val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, destPath) @@ -158,6 +150,7 @@ class TestStructuredStreaming extends HoodieClientTestBase { inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) // When the compaction configs are added, one more commit of the compaction is expected val numExpectedCommits = if (addCompactionConfigs) currNumCommits + 2 else currNumCommits + 1 + waitTillAtleastNCommits(fs, destPath, numExpectedCommits, 120, 5) val commitInstantTime2 = if (tableType == HoodieTableType.MERGE_ON_READ) { @@ -197,8 +190,11 @@ class TestStructuredStreaming extends HoodieClientTestBase { countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect() assertEquals(1, countsPerCommit.length) assertEquals(commitInstantTime2, countsPerCommit(0).get(0)) + + streamingQuery.stop() } - Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf) + + Await.result(f2, Duration("120s")) } @ParameterizedTest @@ -217,7 +213,7 @@ class TestStructuredStreaming extends HoodieClientTestBase { var success = false while ({!success && (currTime - beginTime) < timeoutMsecs}) try { val timeline = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, tablePath) - log.info("Timeline :" + timeline.getInstants.toArray) + log.info("Timeline :" + timeline.getInstants.toArray.mkString("Array(", ", ", ")")) if (timeline.countInstants >= numCommits) { numInstants = timeline.countInstants success = true @@ -254,7 +250,7 @@ class TestStructuredStreaming extends HoodieClientTestBase { @ParameterizedTest @ValueSource(booleans = Array(true, false)) def testStructuredStreamingWithCompaction(isAsyncCompaction: Boolean): Unit = { - structuredStreamingTestRunner(HoodieTableType.MERGE_ON_READ, true, isAsyncCompaction) + structuredStreamingTestRunner(HoodieTableType.MERGE_ON_READ, addCompactionConfigs = true, isAsyncCompaction) } def structuredStreamingForTestClusteringRunner(sourcePath: String, destPath: String, tableType: HoodieTableType, @@ -264,22 +260,25 @@ class TestStructuredStreaming extends HoodieClientTestBase { val records1 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, partitionOfRecords)).toList val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2)) - // Second insert of data - val records2 = recordsToStrings(dataGen.generateInsertsForPartition("001", 100, partitionOfRecords)).toList + // Second batch updates of data + val records2 = recordsToStrings(dataGen.generateInsertsForPartition("000", 100, partitionOfRecords)).toList val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2)) val hudiOptions = getClusteringOpts( tableType, isInlineClustering.toString, isAsyncClustering.toString, "2", 100) - val f1 = initStreamingWriteFuture(inputDF1.schema, sourcePath, destPath, hudiOptions) + + val streamingQuery = initWritingStreamingQuery(inputDF1.schema, sourcePath, destPath, hudiOptions) val f2 = Future { inputDF1.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) // wait for spark streaming to process one microbatch + var currNumCommits = waitTillAtleastNCommits(fs, destPath, 1, 120, 5) assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000")) inputDF2.coalesce(1).write.mode(SaveMode.Append).json(sourcePath) // wait for spark streaming to process second microbatch + currNumCommits = waitTillAtleastNCommits(fs, destPath, currNumCommits + 1, 120, 5) // Wait for the clustering to finish @@ -299,8 +298,11 @@ class TestStructuredStreaming extends HoodieClientTestBase { assertEquals(2, countsPerCommit.length) val commitInstantTime2 = latestInstant(fs, destPath, HoodieTimeline.COMMIT_ACTION) assertEquals(commitInstantTime2, countsPerCommit.maxBy(row => row.getAs[String](0)).get(0)) + + streamingQuery.stop() } - Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf) + + Await.result(f2, Duration("120s")) } private def getLatestFileGroupsFileId(partition: String):Array[String] = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala index 13800e6c8b8c2..90369034e009f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/HoodieSparkSqlTestBase.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.fs.FSUtils +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.exception.ExceptionUtil.getRootCause import org.apache.log4j.Level import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase.checkMessageContains import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.Utils import org.joda.time.DateTimeZone @@ -139,8 +142,11 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll { try { spark.sql(sql) } catch { - case e: Throwable if e.getMessage.trim.contains(errorMsg.trim) => hasException = true - case f: Throwable => fail("Exception should contain: " + errorMsg + ", error message: " + f.getMessage, f) + case e: Throwable if checkMessageContains(e, errorMsg) || checkMessageContains(getRootCause(e), errorMsg) => + hasException = true + + case f: Throwable => + fail("Exception should contain: " + errorMsg + ", error message: " + f.getMessage, f) } assertResult(true)(hasException) } @@ -171,3 +177,19 @@ class HoodieSparkSqlTestBase extends FunSuite with BeforeAndAfterAll { fs.exists(path) } } + +object HoodieSparkSqlTestBase { + + def getLastCommitMetadata(spark: SparkSession, tablePath: String) = { + val metaClient = HoodieTableMetaClient.builder() + .setConf(spark.sparkContext.hadoopConfiguration) + .setBasePath(tablePath) + .build() + + metaClient.getActiveTimeline.getLastCommitMetadataWithValidData.get.getRight + } + + private def checkMessageContains(e: Throwable, text: String): Boolean = + e.getMessage.trim.contains(text.trim) + +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala index e063f67d8c068..9ccc47a57c615 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestAlterTableDropPartition.scala @@ -18,11 +18,17 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ -import org.apache.hudi.HoodieSparkUtils -import org.apache.hudi.common.util.PartitionPathEncodeUtils +import org.apache.hudi.{HoodieCLIUtils, HoodieSparkUtils} +import org.apache.hudi.common.model.HoodieCommitMetadata +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.util.{Option => HOption} +import org.apache.hudi.common.util.{PartitionPathEncodeUtils, StringUtils} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator} import org.apache.spark.sql.SaveMode +import org.junit.jupiter.api.Assertions +import org.junit.jupiter.api.Assertions.assertTrue class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { @@ -336,4 +342,144 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { } } } + + test("check instance schema") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | dt string + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + | partitioned by (dt) + """.stripMargin) + // insert data to table + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000, '01'), " + + s"(2, 'a2', 10, 1000, '02'), (3, 'a3', 10, 1000, '03')") + checkAnswer(s"select id, name, price, ts, dt from $tableName")( + Seq(1, "a1", 10.0, 1000, "01"), + Seq(2, "a2", 10.0, 1000, "02"), + Seq(3, "a3", 10.0, 1000, "03") + ) + + // drop partition + spark.sql(s"alter table $tableName drop partition (dt = '01')") + checkAnswer(s"select id, name, price, ts, dt from $tableName")( + Seq(2, "a2", 10.0, 1000, "02"), + Seq(3, "a3", 10.0, 1000, "03") + ) + + // check schema + val hadoopConf = spark.sessionState.newHadoopConf() + val metaClient = HoodieTableMetaClient.builder().setBasePath(s"${tmp.getCanonicalPath}/$tableName") + .setConf(hadoopConf).build() + val lastInstant = metaClient.getActiveTimeline.lastInstant() + val commitMetadata = HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline.getInstantDetails( + lastInstant.get()).get(), classOf[HoodieCommitMetadata]) + val schemaStr = commitMetadata.getExtraMetadata.get(HoodieCommitMetadata.SCHEMA_KEY) + Assertions.assertFalse(StringUtils.isNullOrEmpty(schemaStr)) + + // delete + spark.sql(s"delete from $tableName where dt = '02'") + checkAnswer(s"select id, name, price, ts, dt from $tableName")( + Seq(3, "a3", 10, 1000, "03") + ) + } + } + } + + test("Prevent a partition from being dropped if there are pending CLUSTERING jobs") { + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}t/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = 'cow', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + | """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + + // Generate the first clustering plan + val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime + client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty()) + + checkAnswer(s"call show_clustering('$tableName')")( + Seq(firstScheduleInstant, 3, HoodieInstant.State.REQUESTED.name(), "*") + ) + + val partition = "ts=1002" + val errMsg = s"Failed to drop partitions. Please ensure that there are no pending table service actions (clustering/compaction) for the partitions to be deleted: [$partition]" + checkExceptionContain(s"ALTER TABLE $tableName DROP PARTITION($partition)")(errMsg) + } + } + + test("Prevent a partition from being dropped if there are pending COMPACTs jobs") { + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}t/$tableName" + // Using INMEMORY index type to ensure that deltacommits generate log files instead of parquet + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = 'mor', + | preCombineField = 'ts', + | hoodie.index.type = 'INMEMORY' + | ) + | partitioned by(ts) + | location '$basePath' + | """.stripMargin) + // Create 5 deltacommits to ensure that it is > default `hoodie.compact.inline.max.delta.commits` + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + + // Generate the first compaction plan + val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime + assertTrue(client.scheduleCompactionAtInstant(firstScheduleInstant, HOption.empty())) + + checkAnswer(s"call show_compaction('$tableName')")( + Seq(firstScheduleInstant, 5, HoodieInstant.State.REQUESTED.name()) + ) + + val partition = "ts=1002" + val errMsg = s"Failed to drop partitions. Please ensure that there are no pending table service actions (clustering/compaction) for the partitions to be deleted: [$partition]" + checkExceptionContain(s"ALTER TABLE $tableName DROP PARTITION($partition)")(errMsg) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala index 6a6b41da7fb73..6240a587e000f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCreateTable.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.HoodieSparkUtils -import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat @@ -27,6 +27,7 @@ import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.hudi.HoodieSparkSqlTestBase.getLastCommitMetadata import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertFalse @@ -298,6 +299,10 @@ class TestCreateTable extends HoodieSparkSqlTestBase { | AS | select 1 as id, 'a1' as name, 10 as price, 1000 as ts """.stripMargin) + + assertResult(WriteOperationType.BULK_INSERT) { + getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName1").getOperationType + } checkAnswer(s"select id, name, price, ts from $tableName1")( Seq(1, "a1", 10.0, 1000) ) @@ -317,6 +322,10 @@ class TestCreateTable extends HoodieSparkSqlTestBase { | select 1 as id, 'a1' as name, 10 as price, '2021-04-01' as dt """.stripMargin ) + + assertResult(WriteOperationType.BULK_INSERT) { + getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName2").getOperationType + } checkAnswer(s"select id, name, price, dt from $tableName2")( Seq(1, "a1", 10, "2021-04-01") ) @@ -355,9 +364,14 @@ class TestCreateTable extends HoodieSparkSqlTestBase { | price """.stripMargin ) + + assertResult(WriteOperationType.BULK_INSERT) { + getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName3").getOperationType + } checkAnswer(s"select id, name, price, cast(dt as string) from $tableName3")( Seq(1, "a1", 10, "2021-05-06 00:00:00") ) + // Create table with date type partition val tableName4 = generateTableName spark.sql( @@ -374,6 +388,10 @@ class TestCreateTable extends HoodieSparkSqlTestBase { | price """.stripMargin ) + + assertResult(WriteOperationType.BULK_INSERT) { + getLastCommitMetadata(spark, s"${tmp.getCanonicalPath}/$tableName4").getOperationType + } checkAnswer(s"select id, name, price, cast(dt as string) from $tableName4")( Seq(1, "a1", 10, "2021-05-06") ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala index 4470712e020c7..b86241eaca955 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.hudi.common.fs.FSUtils +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog @@ -51,6 +52,16 @@ class TestDropTable extends HoodieSparkSqlTestBase { } } + test("Test Drop Table with non existent table") { + // drop table if exists + spark.sql("drop table if exists non_existent_table") + + // drop table + assertThrows[AnalysisException]{ + spark.sql("drop table non_existent_table") + } + } + test("Test Drop Table with purge") { withTempDir { tmp => Seq("cow", "mor").foreach { tableType => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala index 44c23d146c68b..28eb3c7d357d4 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestHoodieOptionConfig.scala @@ -53,7 +53,7 @@ class TestHoodieOptionConfig extends SparkClientFunctionalTestHarness { "hoodie.index.type" -> "INMEMORY", "hoodie.compact.inline" -> "true" ) - val tableConfigs = HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) + val tableConfigs = HoodieOptionConfig.mapSqlOptionsToTableConfigs(sqlOptions) assertTrue(tableConfigs.size == 5) assertTrue(tableConfigs(HoodieTableConfig.RECORDKEY_FIELDS.key) == "id,addr") 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 310f00eefa67d..7d299f5ea063e 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 @@ -22,9 +22,9 @@ import org.apache.hudi.HoodieSparkUtils import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieDuplicateKeyException +import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.internal.SQLConf import java.io.File @@ -993,4 +993,68 @@ class TestInsertTable extends HoodieSparkSqlTestBase { spark.sql("set hoodie.merge.allow.duplicate.on.inserts = false") spark.sql("set hoodie.datasource.write.operation = upsert") } + + /** + * This test is to make sure that bulk insert doesn't create a bunch of tiny files if + * hoodie.bulkinsert.user.defined.partitioner.sort.columns doesn't start with the partition columns + * + * NOTE: Additionally, this test serves as a smoke test making sure that all of the bulk-insert + * modes work + */ + test(s"Test Bulk Insert with all sort-modes") { + withTempDir { basePath => + BulkInsertSortMode.values().foreach { sortMode => + val tableName = generateTableName + // Remove these with [HUDI-5419] + spark.sessionState.conf.unsetConf("hoodie.datasource.write.operation") + spark.sessionState.conf.unsetConf("hoodie.datasource.write.insert.drop.duplicates") + spark.sessionState.conf.unsetConf("hoodie.merge.allow.duplicate.on.inserts") + spark.sessionState.conf.unsetConf("hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled") + // Default parallelism is 200 which means in global sort, each record will end up in a different spark partition so + // 9 files would be created. Setting parallelism to 3 so that each spark partition will contain a hudi partition. + val parallelism = if (sortMode.name.equals(BulkInsertSortMode.GLOBAL_SORT.name())) { + "hoodie.bulkinsert.shuffle.parallelism = 3," + } else { + "" + } + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | dt string + |) using hudi + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'name', + | type = 'cow', + | $parallelism + | hoodie.bulkinsert.sort.mode = '${sortMode.name}' + | ) + | partitioned by (dt) + | location '${basePath.getCanonicalPath}/$tableName' + """.stripMargin) + + spark.sql("set hoodie.sql.bulk.insert.enable = true") + spark.sql("set hoodie.sql.insert.mode = non-strict") + + spark.sql( + s"""insert into $tableName values + |(5, 'a', 35, '2021-05-21'), + |(1, 'a', 31, '2021-01-21'), + |(3, 'a', 33, '2021-03-21'), + |(4, 'b', 16, '2021-05-21'), + |(2, 'b', 18, '2021-01-21'), + |(6, 'b', 17, '2021-03-21'), + |(8, 'a', 21, '2021-05-21'), + |(9, 'a', 22, '2021-01-21'), + |(7, 'a', 23, '2021-03-21') + |""".stripMargin) + + // TODO re-enable + //assertResult(3)(spark.sql(s"select distinct _hoodie_file_name from $tableName").count()) + } + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala index 2d8d6ceca7142..57c82d6bfc072 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestUpdateTable.scala @@ -204,4 +204,38 @@ class TestUpdateTable extends HoodieSparkSqlTestBase { } } } + + test("Test decimal type") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long, + | ff decimal(38, 10) + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | type = 'mor', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000, 10.0") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 10.0, 1000) + ) + + spark.sql(s"update $tableName set price = 22 where id = 1") + checkAnswer(s"select id, name, price, ts from $tableName")( + Seq(1, "a1", 22.0, 1000) + ) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala index 456f9c5066afe..7f63766ee4712 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala @@ -19,11 +19,22 @@ package org.apache.spark.sql.hudi.procedure +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hudi.DataSourceWriteOptions.{OPERATION, RECORDKEY_FIELD} +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType} +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline} import org.apache.hudi.common.util.{Option => HOption} -import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers} - +import org.apache.hudi.common.util.collection.Pair +import org.apache.hudi.config.HoodieClusteringConfig +import org.apache.hudi.{DataSourceReadOptions, HoodieCLIUtils, HoodieDataSourceHelpers, HoodieFileIndex} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.types.{DataTypes, Metadata, StringType, StructField, StructType} +import org.apache.spark.sql.{Dataset, Row} + +import java.util import scala.collection.JavaConverters.asScalaIteratorConverter class TestClusteringProcedure extends HoodieSparkProcedureTestBase { @@ -385,4 +396,120 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { } } } + + test("Test Call run_clustering with partition selected config") { + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = 'cow', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + + // Test clustering with PARTITION_SELECTED config set, choose only a part of all partitions to schedule + { + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1010)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1010)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1011)") + spark.sql(s"set ${HoodieClusteringConfig.PARTITION_SELECTED.key()}=ts=1010") + // Do + val result = spark.sql(s"call run_clustering(table => '$tableName', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(result.length) + assertResult("ts=1010")(result(0)(3)) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1010), + Seq(2, "a2", 10.0, 1010), + Seq(3, "a3", 10.0, 1011) + ) + } + + // Test clustering with PARTITION_SELECTED config set, choose all partitions to schedule + { + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1010)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1011)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1012)") + spark.sql(s"set ${HoodieClusteringConfig.PARTITION_SELECTED.key()}=ts=1010,ts=1011,ts=1012") + val result = spark.sql(s"call run_clustering(table => '$tableName', show_involved_partition => true)") + .collect() + .map(row => Seq(row.getString(0), row.getInt(1), row.getString(2), row.getString(3))) + assertResult(1)(result.length) + assertResult("ts=1010,ts=1011,ts=1012")(result(0)(3)) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1010), + Seq(2, "a2", 10.0, 1010), + Seq(3, "a3", 10.0, 1011), + Seq(4, "a4", 10.0, 1010), + Seq(5, "a5", 10.0, 1011), + Seq(6, "a6", 10.0, 1012) + ) + } + } + } + + def avgRecord(commitTimeline: HoodieTimeline): Long = { + var totalByteSize = 0L + var totalRecordsCount = 0L + commitTimeline.getReverseOrderedInstants.toArray.foreach(instant => { + val commitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(instant.asInstanceOf[HoodieInstant]).get, classOf[HoodieCommitMetadata]) + totalByteSize = totalByteSize + commitMetadata.fetchTotalBytesWritten() + totalRecordsCount = totalRecordsCount + commitMetadata.fetchTotalRecordsWritten() + }) + + Math.ceil((1.0 * totalByteSize) / totalRecordsCount).toLong + } + + def writeRecords(files: Int, numRecords: Int, partitions: Int, location: String, options: Map[String, String]): Unit = { + val records = new util.ArrayList[Row](numRecords) + val rowDimension = Math.ceil(Math.sqrt(numRecords)).toInt + + val data = Stream.range(0, rowDimension, 1) + .flatMap(x => Stream.range(0, rowDimension, 1).map(y => Pair.of(x, y))) + + if (partitions > 0) { + data.foreach { i => + records.add(Row(i.getLeft % partitions, "foo" + i.getLeft, "bar" + i.getRight)) + } + } else { + data.foreach { i => + records.add(Row(i.getLeft, "foo" + i.getLeft, "bar" + i.getRight)) + } + } + + val struct = StructType(Array[StructField]( + StructField("c1", DataTypes.IntegerType, nullable = true, Metadata.empty), + StructField("c2", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("c3", DataTypes.StringType, nullable = true, Metadata.empty) + )) + + // files can not effect for hudi + val df = spark.createDataFrame(records, struct).repartition(files) + writeDF(df, location, options) + } + + def writeDF(df: Dataset[Row], location: String, options: Map[String, String]): Unit = { + df.select("c1", "c2", "c3") + .sortWithinPartitions("c1", "c2") + .write + .format("hudi") + .option(OPERATION.key(), WriteOperationType.INSERT.value()) + .option(RECORDKEY_FIELD.key(), "c1") + .options(options) + .mode("append").save(location) + } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieVectorizedParquetRecordReader.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieVectorizedParquetRecordReader.java new file mode 100644 index 0000000000000..462a993580ed3 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieVectorizedParquetRecordReader.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.execution.datasources.parquet; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hudi.client.utils.SparkInternalSchemaConverter; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector; +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; +import org.apache.spark.sql.execution.vectorized.WritableColumnVector; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class Spark24HoodieVectorizedParquetRecordReader extends VectorizedParquetRecordReader { + + // save the col type change info. + private Map> typeChangeInfos; + + private ColumnarBatch columnarBatch; + + private Map idToColumnVectors; + + private WritableColumnVector[] columnVectors; + + // The capacity of vectorized batch. + private int capacity; + + // If true, this class returns batches instead of rows. + private boolean returnColumnarBatch; + + // The memory mode of the columnarBatch. + private final MemoryMode memoryMode; + + /** + * Batch of rows that we assemble and the current index we've returned. Every time this + * batch is used up (batchIdx == numBatched), we populated the batch. + */ + private int batchIdx = 0; + private int numBatched = 0; + + public Spark24HoodieVectorizedParquetRecordReader( + TimeZone convertTz, + boolean useOffHeap, + int capacity, + Map> typeChangeInfos) { + super(convertTz, useOffHeap, capacity); + memoryMode = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP; + this.typeChangeInfos = typeChangeInfos; + this.capacity = capacity; + } + + @Override + public void initBatch(StructType partitionColumns, InternalRow partitionValues) { + super.initBatch(partitionColumns, partitionValues); + if (columnVectors == null) { + columnVectors = new WritableColumnVector[sparkSchema.length() + partitionColumns.length()]; + } + if (idToColumnVectors == null) { + idToColumnVectors = new HashMap<>(); + typeChangeInfos.entrySet() + .stream() + .forEach(f -> { + WritableColumnVector vector = + memoryMode == MemoryMode.OFF_HEAP ? new OffHeapColumnVector(capacity, f.getValue().getLeft()) : new OnHeapColumnVector(capacity, f.getValue().getLeft()); + idToColumnVectors.put(f.getKey(), vector); + }); + } + } + + @Override + public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException, UnsupportedOperationException { + super.initialize(inputSplit, taskAttemptContext); + } + + @Override + public void close() throws IOException { + super.close(); + for (Map.Entry e : idToColumnVectors.entrySet()) { + e.getValue().close(); + } + idToColumnVectors = null; + columnarBatch = null; + columnVectors = null; + } + + @Override + public ColumnarBatch resultBatch() { + ColumnarBatch currentColumnBatch = super.resultBatch(); + boolean changed = false; + for (Map.Entry> entry : typeChangeInfos.entrySet()) { + boolean rewrite = SparkInternalSchemaConverter + .convertColumnVectorType((WritableColumnVector) currentColumnBatch.column(entry.getKey()), + idToColumnVectors.get(entry.getKey()), currentColumnBatch.numRows()); + if (rewrite) { + changed = true; + columnVectors[entry.getKey()] = idToColumnVectors.get(entry.getKey()); + } + } + if (changed) { + if (columnarBatch == null) { + // fill other vector + for (int i = 0; i < columnVectors.length; i++) { + if (columnVectors[i] == null) { + columnVectors[i] = (WritableColumnVector) currentColumnBatch.column(i); + } + } + columnarBatch = new ColumnarBatch(columnVectors); + } + columnarBatch.setNumRows(currentColumnBatch.numRows()); + return columnarBatch; + } else { + return currentColumnBatch; + } + } + + @Override + public boolean nextBatch() throws IOException { + boolean result = super.nextBatch(); + if (idToColumnVectors != null) { + idToColumnVectors.entrySet().stream().forEach(e -> e.getValue().reset()); + } + numBatched = resultBatch().numRows(); + batchIdx = 0; + return result; + } + + @Override + public void enableReturningBatches() { + returnColumnarBatch = true; + super.enableReturningBatches(); + } + + @Override + public Object getCurrentValue() { + if (typeChangeInfos == null || typeChangeInfos.isEmpty()) { + return super.getCurrentValue(); + } + + if (returnColumnarBatch) { + return columnarBatch == null ? super.getCurrentValue() : columnarBatch; + } + + return columnarBatch == null ? super.getCurrentValue() : columnarBatch.getRow(batchIdx - 1); + } + + @Override + public boolean nextKeyValue() throws IOException { + resultBatch(); + + if (returnColumnarBatch) { + return nextBatch(); + } + + if (batchIdx >= numBatched) { + if (!nextBatch()) { + return false; + } + } + ++batchIdx; + return true; + } +} diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetFileFormat.scala index 6fb5c50c03a2b..1a8585b38aa90 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetFileFormat.scala @@ -31,12 +31,12 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroDeserializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.types.{AtomicType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration import java.net.URI @@ -159,8 +159,17 @@ class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo } val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + + // Clone new conf + val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) + val (implicitTypeChangeInfos, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + + if (!implicitTypeChangeInfos.isEmpty) { + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + val hadoopAttemptContext = - new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) + new TaskAttemptContextImpl(hadoopAttemptConf, attemptId) // Try to push down filters when filter push-down is enabled. // Notice: This push-down is RowGroups level, not individual records. @@ -169,17 +178,29 @@ class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo } val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) + val vectorizedReader = if (!implicitTypeChangeInfos.isEmpty) { + new Spark24HoodieVectorizedParquetRecordReader( + convertTz.orNull, + enableOffHeapColumnVector && taskContext.isDefined, + capacity, + implicitTypeChangeInfos + ) + } else { + new VectorizedParquetRecordReader( + convertTz.orNull, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) + } + val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion lister before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) vectorizedReader.initialize(split, hadoopAttemptContext) - logDebug(s"Appending $partitionSchema ${file.partitionValues}") // NOTE: We're making appending of the partitioned values to the rows read from the // data file configurable if (shouldAppendPartitionValues) { + logDebug(s"Appending $partitionSchema ${file.partitionValues}") vectorizedReader.initBatch(partitionSchema, file.partitionValues) } else { vectorizedReader.initBatch(StructType(Nil), InternalRow.empty) @@ -194,11 +215,12 @@ class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo } else { logDebug(s"Falling back to parquet-mr") // ParquetRecordReader returns UnsafeRow + val readSupport = new ParquetReadSupport(convertTz) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. @@ -206,8 +228,21 @@ class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = if (implicitTypeChangeInfos.isEmpty) { + GenerateUnsafeProjection.generate(fullSchema, fullSchema) + } else { + val newFullSchema = new StructType(requiredSchema.fields.zipWithIndex.map { case (f, i) => + if (implicitTypeChangeInfos.containsKey(i)) { + StructField(f.name, implicitTypeChangeInfos.get(i).getRight, f.nullable, f.metadata) + } else f + }).toAttributes ++ partitionSchema.toAttributes + val castSchema = newFullSchema.zipWithIndex.map { case (attr, i) => + if (implicitTypeChangeInfos.containsKey(i)) { + Cast(attr, implicitTypeChangeInfos.get(i).getLeft) + } else attr + } + GenerateUnsafeProjection.generate(castSchema, newFullSchema) + } // This is a horrible erasure hack... if we type the iterator above, then it actually check // the type in next() and we get a class cast exception. If we make that function return @@ -217,13 +252,14 @@ class Spark24HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo // data file configurable if (!shouldAppendPartitionValues || partitionSchema.length == 0) { // There is no partition columns - iter.asInstanceOf[Iterator[InternalRow]] + iter.asInstanceOf[Iterator[InternalRow]].map(unsafeProjection) } else { + val joinedRow = new JoinedRow() iter.asInstanceOf[Iterator[InternalRow]] - .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + .map(d => unsafeProjection(joinedRow(d, file.partitionValues))) } - } } } + } diff --git a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala index ca41490fc0ac2..9769fc6ca3b6f 100644 --- a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala @@ -151,7 +151,7 @@ class Spark31HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo // Internal schema has to be pruned at this point val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - val shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) val fileSchema = if (shouldUseInternalSchema) { @@ -223,7 +223,8 @@ class Spark31HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo // Clone new conf val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value) - var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { + + val typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) { val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema() val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema) @@ -231,7 +232,12 @@ class Spark31HoodieParquetFileFormat(private val shouldAppendPartitionValues: Bo SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) } else { - new java.util.HashMap() + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + if (!implicitTypeChangeInfo.isEmpty) { + shouldUseInternalSchema = true + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + implicitTypeChangeInfo } val hadoopAttemptContext = diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala index 52d450029e3bd..ae686d33a31b9 100644 --- a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32PlusHoodieParquetFileFormat.scala @@ -146,7 +146,7 @@ class Spark32PlusHoodieParquetFileFormat(private val shouldAppendPartitionValues // Internal schema has to be pruned at this point val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr) - val shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent + var shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) val fileSchema = if (shouldUseInternalSchema) { @@ -228,7 +228,12 @@ class Spark32PlusHoodieParquetFileFormat(private val shouldAppendPartitionValues SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema) } else { - new java.util.HashMap() + val (implicitTypeChangeInfo, sparkRequestSchema) = HoodieParquetFileFormatHelper.buildImplicitSchemaChangeInfo(hadoopAttemptConf, footerFileMetaData, requiredSchema) + if (!implicitTypeChangeInfo.isEmpty) { + shouldUseInternalSchema = true + hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, sparkRequestSchema.json) + } + implicitTypeChangeInfo } val hadoopAttemptContext = @@ -394,7 +399,6 @@ class Spark32PlusHoodieParquetFileFormat(private val shouldAppendPartitionValues } } } - } object Spark32PlusHoodieParquetFileFormat { 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 0d490c4ab14a1..3aebb04827bb0 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 @@ -77,8 +77,6 @@ public HiveSyncConfig(Properties props, Configuration hadoopConf) { super(props, hadoopConf); HiveConf hiveConf = hadoopConf instanceof HiveConf ? (HiveConf) hadoopConf : new HiveConf(hadoopConf, HiveConf.class); - // HiveConf needs to load fs conf to allow instantiation via AWSGlueClientFactory - hiveConf.addResource(getHadoopFileSystem().getConf()); setHadoopConf(hiveConf); validateParameters(); } 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 d0a40bbc181c5..d3ff37ba491b4 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 @@ -36,6 +36,7 @@ import com.beust.jcommander.JCommander; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; @@ -46,6 +47,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; +import static org.apache.hudi.common.util.StringUtils.nonEmpty; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE; @@ -80,15 +83,23 @@ public class HiveSyncTool extends HoodieSyncTool implements AutoCloseable { public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - protected final HiveSyncConfig config; - protected final String databaseName; - protected final String tableName; + private HiveSyncConfig config; + private final String databaseName; + private final String tableName; + protected HoodieSyncClient syncClient; protected String snapshotTableName; protected Option roTableName; public HiveSyncTool(Properties props, Configuration hadoopConf) { super(props, hadoopConf); + String metastoreUris = props.getProperty(METASTORE_URIS.key()); + // Give precedence to HiveConf.ConfVars.METASTOREURIS if it is set. + // Else if user has provided HiveSyncConfigHolder.METASTORE_URIS, then set that in hadoop conf. + if (isNullOrEmpty(hadoopConf.get(HiveConf.ConfVars.METASTOREURIS.varname)) && nonEmpty(metastoreUris)) { + LOG.info(String.format("Setting %s = %s", HiveConf.ConfVars.METASTOREURIS.varname, metastoreUris)); + hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, metastoreUris); + } HiveSyncConfig config = new HiveSyncConfig(props, hadoopConf); this.config = config; this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); @@ -135,7 +146,8 @@ public void syncHoodieTable() { try { if (syncClient != null) { LOG.info("Syncing target hoodie table with hive table(" - + tableId(databaseName, tableName) + "). Hive metastore URL :" + + tableId(databaseName, tableName) + "). Hive metastore URL from HiveConf:" + + config.getHiveConf().get(HiveConf.ConfVars.METASTOREURIS.varname) + "). Hive metastore URL from HiveSyncConfig:" + config.getString(METASTORE_URIS) + ", basePath :" + config.getString(META_SYNC_BASE_PATH)); @@ -174,6 +186,9 @@ public void close() { throw new HoodieHiveSyncException("Fail to close sync client.", e); } } + if (config != null) { + config = null; + } } protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java index 1bdc87ab1123d..cd5e1aba212de 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java @@ -34,6 +34,7 @@ import org.apache.hudi.sync.common.HoodieSyncClient; import org.apache.hudi.sync.common.model.FieldSchema; import org.apache.hudi.sync.common.model.Partition; +import org.apache.hudi.sync.common.util.ConfigUtils; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; @@ -56,6 +57,7 @@ import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; import static org.apache.hudi.sync.common.util.TableUtils.tableId; @@ -299,6 +301,11 @@ public void updateLastCommitTimeSynced(String tableName) { if (lastCommitSynced.isPresent()) { try { Table table = client.getTable(databaseName, tableName); + String basePath = config.getString(META_SYNC_BASE_PATH); + StorageDescriptor sd = table.getSd(); + sd.setLocation(basePath); + SerDeInfo serdeInfo = sd.getSerdeInfo(); + serdeInfo.putToParameters(ConfigUtils.TABLE_SERDE_PATH, basePath); table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get()); client.alter_table(databaseName, tableName, table); } catch (Exception e) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java index c14536a27743b..e93e42a9874bf 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java @@ -128,12 +128,12 @@ public void createTable(String tableName, MessageType storageSchema, String inpu if (!syncConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)) { newTb.putToParameters("EXTERNAL", "TRUE"); + newTb.setTableType(TableType.EXTERNAL_TABLE.toString()); } for (Map.Entry entry : tableProperties.entrySet()) { newTb.putToParameters(entry.getKey(), entry.getValue()); } - newTb.setTableType(TableType.EXTERNAL_TABLE.toString()); client.createTable(newTb); } catch (Exception e) { LOG.error("failed to create table " + tableName, e); 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 acd75595fb887..897570e159796 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.hive.testutils.HiveTestUtil; import org.apache.hudi.sync.common.model.FieldSchema; import org.apache.hudi.sync.common.model.Partition; @@ -47,6 +48,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.net.URISyntaxException; @@ -308,6 +310,32 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, } } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSyncCOWTableWithCreateManagedTable(boolean createManagedTable) throws Exception { + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), HiveSyncMode.HMS.name()); + hiveSyncProps.setProperty(HIVE_CREATE_MANAGED_TABLE.key(), Boolean.toString(createManagedTable)); + + String instantTime = "100"; + HiveTestUtil.createCOWTable(instantTime, 5, true); + + reinitHiveSyncClient(); + reSyncHiveTable(); + + SessionState.start(HiveTestUtil.getHiveConf()); + Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); + hiveDriver.run(String.format("SHOW TBLPROPERTIES %s.%s", HiveTestUtil.DB_NAME, HiveTestUtil.TABLE_NAME)); + List results = new ArrayList<>(); + hiveDriver.getResults(results); + + assertEquals( + String.format("%slast_commit_time_sync\t%s\n%s", + createManagedTable ? StringUtils.EMPTY_STRING : "EXTERNAL\tTRUE\n", + instantTime, + getSparkTableProperties(true, true)), + String.format("%s\n", String.join("\n", results.subList(0, results.size() - 1)))); + } + private String getSparkTableProperties(boolean syncAsDataSourceTable, boolean useSchemaFromCommitMetadata) { if (syncAsDataSourceTable) { if (useSchemaFromCommitMetadata) { diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java index af06f5908ce39..d73bf2ede249c 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java @@ -88,10 +88,9 @@ public boolean isBootstrap() { * Going through archive timeline is a costly operation, and it should be avoided unless some start time is given. */ public Set getDroppedPartitionsSince(Option lastCommitTimeSynced) { - HoodieTimeline timeline = lastCommitTimeSynced.isPresent() ? metaClient.getArchivedTimeline(lastCommitTimeSynced.get()) - .mergeTimeline(metaClient.getActiveTimeline()) - .getCommitsTimeline() - .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE) : metaClient.getActiveTimeline(); + HoodieTimeline timeline = lastCommitTimeSynced.isPresent() + ? TimelineUtils.getCommitsTimelineAfter(metaClient, lastCommitTimeSynced.get()) + : metaClient.getActiveTimeline(); return new HashSet<>(TimelineUtils.getDroppedPartitions(timeline)); } @@ -117,10 +116,7 @@ public List getWrittenPartitionsSince(Option lastCommitTimeSynce } else { LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); return TimelineUtils.getWrittenPartitions( - metaClient.getArchivedTimeline(lastCommitTimeSynced.get()) - .mergeTimeline(metaClient.getActiveTimeline()) - .getCommitsTimeline() - .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE)); + TimelineUtils.getCommitsTimelineAfter(metaClient, lastCommitTimeSynced.get())); } } diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java index 62cd4c1748fcd..8d04ca8ad7c25 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java @@ -33,6 +33,12 @@ public class ConfigUtils { */ public static final String IS_QUERY_AS_RO_TABLE = "hoodie.query.as.ro.table"; + /** + * Config stored in hive serde properties to tell query engine (spark) the + * location to read. + */ + public static final String TABLE_SERDE_PATH = "path"; + /** * Convert the key-value config to a map.The format of the config * is a key-value pair just like "k1=v1\nk2=v2\nk3=v3". diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java index 31c845397d700..2a9f783a43bc3 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java @@ -98,8 +98,9 @@ public static Map getSparkTableProperties(List partition public static Map getSparkSerdeProperties(boolean readAsOptimized, String basePath) { Map sparkSerdeProperties = new HashMap<>(); - sparkSerdeProperties.put("path", basePath); + sparkSerdeProperties.put(ConfigUtils.TABLE_SERDE_PATH, basePath); sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized)); return sparkSerdeProperties; } + } diff --git a/hudi-tests-common/src/main/resources/log4j2-surefire.properties b/hudi-tests-common/src/main/resources/log4j2-surefire.properties index 6b6b2fa5e5322..e1c8223612893 100644 --- a/hudi-tests-common/src/main/resources/log4j2-surefire.properties +++ b/hudi-tests-common/src/main/resources/log4j2-surefire.properties @@ -32,6 +32,6 @@ rootLogger.appenderRef.stdout.ref = CONSOLE logger.apache.name = org.apache logger.apache.level = info logger.hudi.name = org.apache.hudi -logger.hudi.level = debug +logger.hudi.level = info logger.hbase.name = org.apache.hadoop.hbase logger.hbase.level = error \ No newline at end of file diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index aab95d8bdc6a1..7dd04b85c578f 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -56,6 +56,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; @@ -181,19 +182,20 @@ private boolean isLocalViewBehind(Context ctx) { * Syncs data-set view if local view is behind. */ private boolean syncIfLocalViewBehind(Context ctx) { - if (isLocalViewBehind(ctx)) { - String basePath = ctx.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM); - String lastKnownInstantFromClient = ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, String.class).getOrDefault(HoodieTimeline.INVALID_INSTANT_TS); - SyncableFileSystemView view = viewManager.getFileSystemView(basePath); - synchronized (view) { - if (isLocalViewBehind(ctx)) { - HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline(); - LOG.info("Syncing view as client passed last known instant " + lastKnownInstantFromClient - + " as last known instant but server has the following last instant on timeline :" - + localTimeline.lastInstant()); - view.sync(); - return true; - } + String basePath = ctx.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM); + SyncableFileSystemView view = viewManager.getFileSystemView(basePath); + synchronized (view) { + if (isLocalViewBehind(ctx)) { + + String lastKnownInstantFromClient = ctx.queryParamAsClass( + RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, String.class) + .getOrDefault(HoodieTimeline.INVALID_INSTANT_TS); + HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline(); + LOG.info("Syncing view as client passed last known instant " + lastKnownInstantFromClient + + " as last known instant but server has the following last instant on timeline :" + + localTimeline.lastInstant()); + view.sync(); + return true; } } return false; @@ -277,6 +279,14 @@ private void registerDataFilesAPI() { writeValueAsString(ctx, dtos); }, true)); + app.get(RemoteHoodieTableFileSystemView.ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> { + metricsRegistry.add("ALL_LATEST_BASE_FILES_BEFORE_ON_INSTANT", 1); + Map> dtos = dataFileHandler.getAllLatestDataFilesBeforeOrOn( + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.BASEPATH_PARAM, String.class).getOrThrow(e -> new HoodieException("Basepath is invalid")), + ctx.queryParamAsClass(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM, String.class).getOrThrow(e -> new HoodieException("MAX_INSTANT_PARAM is invalid"))); + writeValueAsString(ctx, dtos); + }, true)); + app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILE_ON_INSTANT_URL, new ViewHandler(ctx -> { metricsRegistry.add("LATEST_DATA_FILE_ON_INSTANT", 1); List dtos = dataFileHandler.getLatestDataFileOn( diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java index 5033a86f8f2e4..a34b49843fac1 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/BaseFileHandler.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; /** @@ -61,6 +62,16 @@ public List getLatestDataFilesBeforeOrOn(String basePath, String pa .map(BaseFileDTO::fromHoodieBaseFile).collect(Collectors.toList()); } + public Map> getAllLatestDataFilesBeforeOrOn(String basePath, String maxInstantTime) { + return viewManager.getFileSystemView(basePath) + .getAllLatestBaseFilesBeforeOrOn(maxInstantTime) + .entrySet().stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> entry.getValue().map(BaseFileDTO::fromHoodieBaseFile).collect(Collectors.toList()) + )); + } + public List getLatestDataFileOn(String basePath, String partitionPath, String instantTime, String fileId) { List result = new ArrayList<>(); 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 f6905f92d9440..819ec9bceb738 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 @@ -54,7 +54,7 @@ public class HoodieClusteringJob { private final Config cfg; private final TypedProperties props; private final JavaSparkContext jsc; - private final HoodieTableMetaClient metaClient; + private HoodieTableMetaClient metaClient; public HoodieClusteringJob(JavaSparkContext jsc, Config cfg) { this.cfg = cfg; @@ -180,6 +180,7 @@ public int cluster(int retry) { } private int doCluster(JavaSparkContext jsc) throws Exception { + metaClient = HoodieTableMetaClient.reload(metaClient); String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { if (StringUtils.isNullOrEmpty(cfg.clusteringInstantTime)) { @@ -208,6 +209,7 @@ public Option doSchedule() throws Exception { } private Option doSchedule(JavaSparkContext jsc) throws Exception { + metaClient = HoodieTableMetaClient.reload(metaClient); String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { return doSchedule(client); @@ -224,6 +226,7 @@ private Option doSchedule(SparkRDDWriteClient clien private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { LOG.info("Step 1: Do schedule"); + metaClient = HoodieTableMetaClient.reload(metaClient); String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { Option instantTime = Option.empty(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index d1d98475eae1f..89b8a6a261675 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -30,10 +30,12 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -79,8 +81,10 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -89,18 +93,19 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME; +import static org.apache.hudi.hadoop.CachingPath.getPathWithoutSchemeAndAuthority; /** * A validator with spark-submit to compare information, such as partitions, file listing, index, etc., * between metadata table and filesystem. *

* There are five validation tasks, that can be enabled independently through the following CLI options: - * - `--validate-latest-file-slices`: validate latest file slices for all partitions. - * - `--validate-latest-base-files`: validate latest base files for all partitions. + * - `--validate-latest-file-slices`: validate the latest file slices for all partitions. + * - `--validate-latest-base-files`: validate the latest base files for all partitions. * - `--validate-all-file-groups`: validate all file groups, and all file slices within file groups. * - `--validate-all-column-stats`: validate column stats for all columns in the schema * - `--validate-bloom-filters`: validate bloom filters of base files - * + *

* If the Hudi table is on the local file system, the base path passed to `--base-path` must have * "file:" prefix to avoid validation failure. *

@@ -109,37 +114,36 @@ * Example command: * ``` * spark-submit \ - * --class org.apache.hudi.utilities.HoodieMetadataTableValidator \ - * --master spark://xxxx:7077 \ - * --driver-memory 1g \ - * --executor-memory 1g \ - * $HUDI_DIR/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.11.0-SNAPSHOT.jar \ - * --base-path basePath \ - * --validate-latest-file-slices \ - * --validate-latest-base-files \ - * --validate-all-file-groups + * --class org.apache.hudi.utilities.HoodieMetadataTableValidator \ + * --master spark://xxxx:7077 \ + * --driver-memory 1g \ + * --executor-memory 1g \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.13.0-SNAPSHOT.jar \ + * --base-path basePath \ + * --validate-latest-file-slices \ + * --validate-latest-base-files \ + * --validate-all-file-groups * ``` * *

- * Also You can set `--continuous` for long running this validator. + * Also, You can set `--continuous` for long running this validator. * And use `--min-validate-interval-seconds` to control the validation frequency, default is 10 minutes. *

* Example command: * ``` * spark-submit \ - * --class org.apache.hudi.utilities.HoodieMetadataTableValidator \ - * --master spark://xxxx:7077 \ - * --driver-memory 1g \ - * --executor-memory 1g \ - * $HUDI_DIR/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.11.0-SNAPSHOT.jar \ - * --base-path basePath \ - * --validate-latest-file-slices \ - * --validate-latest-base-files \ - * --validate-all-file-groups \ - * --continuous \ - * --min-validate-interval-seconds 60 + * --class org.apache.hudi.utilities.HoodieMetadataTableValidator \ + * --master spark://xxxx:7077 \ + * --driver-memory 1g \ + * --executor-memory 1g \ + * $HUDI_DIR/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.13.0-SNAPSHOT.jar \ + * --base-path basePath \ + * --validate-latest-file-slices \ + * --validate-latest-base-files \ + * --validate-all-file-groups \ + * --continuous \ + * --min-validate-interval-seconds 60 * ``` - * */ public class HoodieMetadataTableValidator implements Serializable { @@ -751,6 +755,8 @@ private void validateFileSlices( if (fileSliceListFromMetadataTable.size() != fileSliceListFromFS.size()) { mismatch = true; } else if (!fileSliceListFromMetadataTable.equals(fileSliceListFromFS)) { + // In-memory cache for the set of committed files of commits of interest + Map> committedFilesMap = new HashMap<>(); for (int i = 0; i < fileSliceListFromMetadataTable.size(); i++) { FileSlice fileSlice1 = fileSliceListFromMetadataTable.get(i); FileSlice fileSlice2 = fileSliceListFromFS.get(i); @@ -760,7 +766,8 @@ private void validateFileSlices( mismatch = true; break; } - if (!areFileSliceCommittedLogFilesMatching(fileSlice1, fileSlice2, metaClient)) { + if (!areFileSliceCommittedLogFilesMatching( + fileSlice1, fileSlice2, metaClient, committedFilesMap)) { mismatch = true; break; } else { @@ -784,13 +791,17 @@ private void validateFileSlices( /** * Compares committed log files from two file slices. * - * @param fs1 File slice 1 - * @param fs2 File slice 2 - * @param metaClient {@link HoodieTableMetaClient} instance + * @param fs1 File slice 1 + * @param fs2 File slice 2 + * @param metaClient {@link HoodieTableMetaClient} instance + * @param committedFilesMap In-memory map for caching committed files of commits * @return {@code true} if matching; {@code false} otherwise. */ private boolean areFileSliceCommittedLogFilesMatching( - FileSlice fs1, FileSlice fs2, HoodieTableMetaClient metaClient) { + FileSlice fs1, + FileSlice fs2, + HoodieTableMetaClient metaClient, + Map> committedFilesMap) { Set fs1LogPathSet = fs1.getLogFiles().map(f -> f.getPath().toString()).collect(Collectors.toSet()); Set fs2LogPathSet = @@ -803,26 +814,31 @@ private boolean areFileSliceCommittedLogFilesMatching( // Check if the remaining log files are uncommitted. If there is any log file // that is committed, the committed log files of two file slices are different FileSystem fileSystem = metaClient.getFs(); - HoodieTimeline commitsTimeline = metaClient.getCommitsTimeline(); - if (hasCommittedLogFiles(fileSystem, fs1LogPathSet, commitsTimeline)) { - LOG.error("The first file slice has committed log files that cause mismatching: " - + fs1); + + if (hasCommittedLogFiles(fileSystem, fs1LogPathSet, metaClient, committedFilesMap)) { + LOG.error("The first file slice has committed log files that cause mismatching: " + fs1 + + "; Different log files are: " + fs1LogPathSet); return false; } - if (hasCommittedLogFiles(fileSystem, fs2LogPathSet, commitsTimeline)) { - LOG.error("The second file slice has committed log files that cause mismatching: " - + fs2); + if (hasCommittedLogFiles(fileSystem, fs2LogPathSet, metaClient, committedFilesMap)) { + LOG.error("The second file slice has committed log files that cause mismatching: " + fs2 + + "; Different log files are: " + fs2LogPathSet); return false; } return true; } private boolean hasCommittedLogFiles( - FileSystem fs, Set logFilePathSet, HoodieTimeline commitsTimeline) { + FileSystem fs, + Set logFilePathSet, + HoodieTableMetaClient metaClient, + Map> committedFilesMap) { if (logFilePathSet.isEmpty()) { return false; } + String basePath = metaClient.getBasePathV2().toString(); + HoodieTimeline commitsTimeline = metaClient.getCommitsTimeline(); AvroSchemaConverter converter = new AvroSchemaConverter(); HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants(); HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights(); @@ -844,13 +860,56 @@ private boolean hasCommittedLogFiles( if (reader.hasNext()) { HoodieLogBlock block = reader.next(); final String instantTime = block.getLogBlockHeader().get(INSTANT_TIME); - if (!completedInstantsTimeline.containsOrBeforeTimelineStarts(instantTime) - || inflightInstantsTimeline.containsInstant(instantTime)) { + if (completedInstantsTimeline.containsInstant(instantTime)) { + // The instant is completed, in active timeline + // Checking commit metadata only as log files can only be written by COMMIT or DELTA_COMMIT + if (!committedFilesMap.containsKey(instantTime)) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + completedInstantsTimeline.getInstantDetails( + completedInstantsTimeline.filter(i -> i.getTimestamp().equals(instantTime)) + .firstInstant().get() + ).get(), + HoodieCommitMetadata.class + ); + committedFilesMap.put( + instantTime, + commitMetadata.getWriteStats().stream() + .map(HoodieWriteStat::getPath).collect(Collectors.toSet()) + ); + } + + // Here we check if the commit metadata contains the log file. + // Note that, a log file may be written by a successful write transaction + // leading to a delta commit, but such a log file can be uncommitted and + // not be part of any snapshot, due to Spark task retries for example. + // In such a case, the log file can stay in the file system, but the metadata + // table does not contain the log file for file listing, which is an expected + // behavior. + String relativeLogFilePathStr = getRelativePath(basePath, logFilePathStr); + if (committedFilesMap.get(instantTime).contains(relativeLogFilePathStr)) { + LOG.warn("Log file is committed in an instant in active timeline: instantTime=" + + instantTime + " " + logFilePathStr); + return true; + } else { + LOG.warn("Log file is uncommitted in a completed instant, likely due to retry: " + + "instantTime=" + instantTime + " " + logFilePathStr); + } + } else if (completedInstantsTimeline.isBeforeTimelineStarts(instantTime)) { + // The instant is in archived timeline + LOG.warn("Log file is committed in an instant in archived timeline: instantTime=" + + instantTime + " " + logFilePathStr); + return true; + } else if (inflightInstantsTimeline.containsInstant(instantTime)) { + // The instant is inflight in active timeline // hit an uncommitted block possibly from a failed write - LOG.warn("Log file is uncommitted: " + logFilePathStr); + LOG.warn("Log file is uncommitted because of an inflight instant: instantTime=" + + instantTime + " " + logFilePathStr); } else { - LOG.warn("Log file is committed: " + logFilePathStr); - return true; + // The instant is after the start of the active timeline, + // but it cannot be found in the active timeline + LOG.warn("Log file is uncommitted because the instant is after the start of the " + + "active timeline but absent or in requested in the active timeline: instantTime=" + + instantTime + " " + logFilePathStr); } } else { LOG.warn("There is no log block in " + logFilePathStr); @@ -866,6 +925,19 @@ private boolean hasCommittedLogFiles( return false; } + private String getRelativePath(String basePath, String absoluteFilePath) { + String basePathStr = getPathWithoutSchemeAndAuthority(new Path(basePath)).toString(); + String absoluteFilePathStr = getPathWithoutSchemeAndAuthority(new Path(absoluteFilePath)).toString(); + + if (!absoluteFilePathStr.startsWith(basePathStr)) { + throw new IllegalArgumentException("File path does not belong to the base path! basePath=" + + basePathStr + " absoluteFilePathStr=" + absoluteFilePathStr); + } + + String relativePathStr = absoluteFilePathStr.substring(basePathStr.length()); + return relativePathStr.startsWith("/") ? relativePathStr.substring(1) : relativePathStr; + } + public class AsyncMetadataTableValidateService extends HoodieAsyncService { private final transient ExecutorService executor = Executors.newSingleThreadExecutor(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index 834938a3179dc..bfee08fdb958d 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -175,7 +175,12 @@ private static TypedProperties combineProperties(Config cfg, Option ds.shutdown(false)); + deltaSyncService.ifPresent(ds -> { + LOG.info("Shutting down DeltaStreamer"); + ds.shutdown(false); + LOG.info("Async service shutdown complete. Closing DeltaSync "); + ds.close(); + }); } /** diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java index 7b8232f6194a3..53865e7dec8fc 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java @@ -32,7 +32,6 @@ import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper; -import com.esotericsoftware.minlog.Log; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -109,7 +108,7 @@ private DataFrameReader getDataFrameReader(String fileFormat) { } catch (IOException e) { throw new HoodieException(String.format("Failed to parse sparkOptions: %s", props.getString(Config.SPARK_DATASOURCE_OPTIONS)), e); } - Log.info(String.format("sparkOptions loaded: %s", sparkOptionsMap)); + LOG.info(String.format("sparkOptions loaded: %s", sparkOptionsMap)); dataFrameReader = dataFrameReader.options(sparkOptionsMap); } return dataFrameReader; diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java index 00cf3ae8832b2..607d6dfe284bb 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieRepairTool.java @@ -80,6 +80,7 @@ public class TestHoodieRepairTool extends HoodieCommonTestHarness implements Spa // instant time -> partitionPathToFileIdAndNameMap private final Map>>> instantInfoMap = new HashMap<>(); private final List allFileAbsolutePathList = new ArrayList<>(); + private String tableBasePath; private java.nio.file.Path backupTempDir; @BeforeAll @@ -101,18 +102,20 @@ public void initWithCleanState() throws IOException { context = new HoodieSparkEngineContext(jsc); } initPath(); - metaClient = HoodieTestUtils.init(basePath, getTableType()); + java.nio.file.Path basePath = tempDir.resolve("dataset"); + this.tableBasePath = basePath.toUri().toString(); + metaClient = HoodieTestUtils.init(tableBasePath, getTableType()); backupTempDir = tempDir.resolve("backup"); cleanUpDanglingDataFilesInFS(); cleanUpBackupTempDir(); HoodieTestCommitGenerator.setupTimelineInFS( - basePath, BASE_FILE_INFO, LOG_FILE_INFO, instantInfoMap); + tableBasePath, BASE_FILE_INFO, LOG_FILE_INFO, instantInfoMap); allFileAbsolutePathList.clear(); allFileAbsolutePathList.addAll(instantInfoMap.entrySet().stream() .flatMap(e -> e.getValue().entrySet().stream() .flatMap(partition -> partition.getValue().stream() .map(fileInfo -> new Path( - new Path(basePath, partition.getKey()), fileInfo.getValue()).toString()) + new Path(tableBasePath, partition.getKey()), fileInfo.getValue()).toString()) .collect(Collectors.toList()) .stream()) .collect(Collectors.toList()) @@ -139,7 +142,7 @@ private void cleanUpDanglingDataFilesInFS() { FileSystem fs = metaClient.getFs(); DANGLING_DATA_FILE_LIST.forEach( relativeFilePath -> { - Path path = new Path(basePath, relativeFilePath); + Path path = new Path(tableBasePath, relativeFilePath); try { if (fs.exists(path)) { fs.delete(path, false); @@ -167,8 +170,8 @@ private static void initDanglingDataFileList() { private Stream configPathParams() { Object[][] data = new Object[][] { - {null, basePath, -1}, {basePath + "/backup", basePath, -1}, - {"/tmp/backup", basePath, 0} + {null, tableBasePath, -1}, {tableBasePath + "/backup", tableBasePath, -1}, + {"/tmp/backup", tableBasePath, 0} }; return Stream.of(data).map(Arguments::of); } @@ -178,12 +181,12 @@ public void testCheckBackupPathAgainstBasePath() { configPathParams().forEach(arguments -> { Object[] args = arguments.get(); String backupPath = (String) args[0]; - String basePath = (String) args[1]; + String tableBasePath = (String) args[1]; int expectedResult = (Integer) args[2]; HoodieRepairTool.Config config = new HoodieRepairTool.Config(); config.backupPath = backupPath; - config.basePath = basePath; + config.basePath = tableBasePath; HoodieRepairTool tool = new HoodieRepairTool(jsc, config); assertEquals(expectedResult, tool.checkBackupPathAgainstBasePath()); }); @@ -198,9 +201,9 @@ private Stream configPathParamsWithFS() throws IOException { FSUtils.createPathIfNotExists(metaClient.getFs(), new Path(nonEmptyBackupPath)); FSUtils.createPathIfNotExists(metaClient.getFs(), new Path(nonEmptyBackupPath, ".hoodie")); Object[][] data = new Object[][] { - {null, basePath, 0}, {"/tmp/backup", basePath, 0}, - {emptyBackupPath, basePath, 0}, {basePath + "/backup", basePath, -1}, - {nonEmptyBackupPath, basePath, -1}, + {null, tableBasePath, 0}, {"/tmp/backup", tableBasePath, 0}, + {emptyBackupPath, tableBasePath, 0}, {tableBasePath + "/backup", tableBasePath, -1}, + {nonEmptyBackupPath, tableBasePath, -1}, }; return Stream.of(data).map(Arguments::of); } @@ -210,12 +213,12 @@ public void testCheckBackupPathForRepair() throws IOException { for (Arguments arguments: configPathParamsWithFS().collect(Collectors.toList())) { Object[] args = arguments.get(); String backupPath = (String) args[0]; - String basePath = (String) args[1]; + String tableBasePath = (String) args[1]; int expectedResult = (Integer) args[2]; HoodieRepairTool.Config config = new HoodieRepairTool.Config(); config.backupPath = backupPath; - config.basePath = basePath; + config.basePath = tableBasePath; HoodieRepairTool tool = new HoodieRepairTool(jsc, config); assertEquals(expectedResult, tool.checkBackupPathForRepair()); if (backupPath == null) { @@ -236,7 +239,7 @@ public void testRepairWithIntactInstants() throws IOException { @Test public void testRepairWithBrokenInstants() throws IOException { - List tableDanglingFileList = createDanglingDataFilesInFS(basePath); + List tableDanglingFileList = createDanglingDataFilesInFS(tableBasePath); String backupPath = backupTempDir.toAbsolutePath().toString(); List backupDanglingFileList = DANGLING_DATA_FILE_LIST.stream() .map(filePath -> new Path(backupPath, filePath).toString()) @@ -252,7 +255,7 @@ public void testRepairWithBrokenInstants() throws IOException { @Test public void testRepairWithOneBrokenInstant() throws IOException { - List tableDanglingFileList = createDanglingDataFilesInFS(basePath); + List tableDanglingFileList = createDanglingDataFilesInFS(tableBasePath); String backupPath = backupTempDir.toAbsolutePath().toString(); List backupDanglingFileList = DANGLING_DATA_FILE_LIST .subList(1, 2).stream() @@ -270,7 +273,7 @@ public void testRepairWithOneBrokenInstant() throws IOException { @Test public void testDryRunWithBrokenInstants() throws IOException { - List tableDanglingFileList = createDanglingDataFilesInFS(basePath); + List tableDanglingFileList = createDanglingDataFilesInFS(tableBasePath); String backupPath = backupTempDir.toAbsolutePath().toString(); List backupDanglingFileList = DANGLING_DATA_FILE_LIST.stream() .map(filePath -> new Path(backupPath, filePath).toString()) @@ -286,7 +289,7 @@ public void testDryRunWithBrokenInstants() throws IOException { @Test public void testDryRunWithOneBrokenInstant() throws IOException { - List tableDanglingFileList = createDanglingDataFilesInFS(basePath); + List tableDanglingFileList = createDanglingDataFilesInFS(tableBasePath); String backupPath = backupTempDir.toAbsolutePath().toString(); List backupDanglingFileList = DANGLING_DATA_FILE_LIST.stream() .map(filePath -> new Path(backupPath, filePath).toString()) @@ -313,26 +316,36 @@ public void testUndoWithNonExistentBackupPath() throws IOException { @Test public void testUndoWithExistingBackupPath() throws IOException { - String backupPath = backupTempDir.toAbsolutePath().toString(); + FileSystem fs = FSUtils.getFs(tableBasePath, jsc.hadoopConfiguration()); + java.nio.file.Files.createDirectories(backupTempDir); + String backupPath = backupTempDir.toUri().toString(); + LOG.warn("XXX 111 Base path " + tableBasePath + " : " + fs.getScheme()); + LOG.warn("XXX 111 backup path " + backupPath + ", is exists " + fs.exists(new Path(backupPath)) + " : " + fs.getScheme()); List backupDanglingFileList = createDanglingDataFilesInFS(backupPath); + LOG.warn("XXX 222 Created danglign data files "); List restoreDanglingFileList = DANGLING_DATA_FILE_LIST.stream() - .map(filePath -> new Path(basePath, filePath).toString()) + .map(filePath -> new Path(tableBasePath, filePath).toString()) .collect(Collectors.toList()); List existingFileList = new ArrayList<>(allFileAbsolutePathList); existingFileList.addAll(backupDanglingFileList); existingFileList.addAll(restoreDanglingFileList); + LOG.warn("XXX 333 Verifying files in FS "); verifyFilesInFS(allFileAbsolutePathList, restoreDanglingFileList); + LOG.warn("XXX 444 Verifying files in FS "); verifyFilesInFS(backupDanglingFileList, Collections.emptyList()); + LOG.warn("XXX 555 repair 1"); testRepairToolWithMode( Option.empty(), Option.empty(), HoodieRepairTool.Mode.UNDO.toString(), backupPath, true, existingFileList, Collections.emptyList()); + LOG.warn("XXX 555 repair 2"); // Second run should fail testRepairToolWithMode( Option.empty(), Option.empty(), HoodieRepairTool.Mode.UNDO.toString(), backupPath, false, existingFileList, Collections.emptyList()); + LOG.warn("XXX 666 repair"); } private void testRepairToolWithMode( @@ -341,7 +354,7 @@ private void testRepairToolWithMode( List existFilePathList, List nonExistFilePathList) throws IOException { HoodieRepairTool.Config config = new HoodieRepairTool.Config(); config.backupPath = backupPath; - config.basePath = basePath; + config.basePath = tableBasePath; config.assumeDatePartitioning = true; if (startingInstantOption.isPresent()) { config.startingInstantTime = startingInstantOption.get(); 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 47dff4ef43862..88632b26636d3 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 @@ -26,6 +26,7 @@ import org.apache.hudi.common.config.DFSPropertiesConfiguration; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -164,10 +165,16 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase { private static final Logger LOG = LogManager.getLogger(TestHoodieDeltaStreamer.class); protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster) throws IOException { - HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT); + return initialHoodieDeltaStreamer(tableBasePath, totalRecords, asyncCluster, WriteOperationType.INSERT); + } + + protected HoodieDeltaStreamer initialHoodieDeltaStreamer(String tableBasePath, int totalRecords, String asyncCluster, + WriteOperationType writeOperationType) throws IOException { + HoodieDeltaStreamer.Config cfg = TestHelpers.makeConfig(tableBasePath, writeOperationType); cfg.continuousMode = true; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "", "", asyncCluster, "")); + cfg.configs.addAll(getAllMultiWriterConfigs()); return new HoodieDeltaStreamer(cfg, jsc); } @@ -178,6 +185,7 @@ protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, S protected HoodieClusteringJob initialHoodieClusteringJob(String tableBasePath, String clusteringInstantTime, Boolean runSchedule, String scheduleAndExecute, Boolean retryLastFailedClusteringJob) { HoodieClusteringJob.Config scheduleClusteringConfig = buildHoodieClusteringUtilConfig(tableBasePath, clusteringInstantTime, runSchedule, scheduleAndExecute, retryLastFailedClusteringJob); + scheduleClusteringConfig.configs.addAll(getAllMultiWriterConfigs()); return new HoodieClusteringJob(jsc, scheduleClusteringConfig); } @@ -954,7 +962,6 @@ public void testDeltaSyncWithPendingClustering() throws Exception { @ValueSource(booleans = {true, false}) public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws Exception { String tableBasePath = dfsBasePath + "/cleanerDeleteReplacedDataWithArchive" + asyncClean; - int totalRecords = 3000; // Step 1 : Prepare and insert data without archival and cleaner. @@ -1018,7 +1025,7 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws configs.add(String.format("%s=%s", HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "3")); configs.add(String.format("%s=%s", HoodieCleanConfig.ASYNC_CLEAN.key(), asyncClean)); configs.add(String.format("%s=%s", HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key(), "1")); - cfg.configs.add(String.format("%s=%s", HoodieWriteConfig.MARKERS_TYPE.key(), "DIRECT")); + configs.add(String.format("%s=%s", HoodieWriteConfig.MARKERS_TYPE.key(), "DIRECT")); if (asyncClean) { configs.add(String.format("%s=%s", HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())); @@ -1071,6 +1078,15 @@ private List getAsyncServicesConfigs(int totalRecords, String autoClean, return configs; } + private List getAllMultiWriterConfigs() { + List configs = new ArrayList<>(); + configs.add(String.format("%s=%s", HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getCanonicalName())); + configs.add(String.format("%s=%s", LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,"3000")); + configs.add(String.format("%s=%s", HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())); + configs.add(String.format("%s=%s", HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY.key(), HoodieFailedWritesCleaningPolicy.LAZY.name())); + return configs; + } + private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePath, String clusteringInstantTime, Boolean runSchedule) { @@ -1265,6 +1281,7 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob cfg.continuousMode = false; cfg.tableType = HoodieTableType.COPY_ON_WRITE.name(); cfg.configs.addAll(getAsyncServicesConfigs(totalRecords, "false", "false", "0", "false", "0")); + cfg.configs.addAll(getAllMultiWriterConfigs()); HoodieDeltaStreamer ds = new HoodieDeltaStreamer(cfg, jsc); ds.sync(); @@ -1304,9 +1321,8 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob @ValueSource(strings = {"execute", "schedule", "scheduleAndExecute"}) public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMode) throws Exception { String tableBasePath = dfsBasePath + "/asyncClustering2"; - HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false"); + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 3000, "false", WriteOperationType.BULK_INSERT); HoodieClusteringJob scheduleClusteringJob = initialHoodieClusteringJob(tableBasePath, null, true, runningMode); - deltaStreamerTestRunner(ds, (r) -> { Exception exception = null; TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); @@ -1315,7 +1331,7 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod if (result == 0) { LOG.info("Cluster success"); } else { - LOG.warn("Import failed"); + LOG.warn("Cluster failed"); if (!runningMode.toLowerCase().equals(EXECUTE)) { return false; } diff --git a/packaging/bundle-validation/spark_hadoop_mr/write.scala b/packaging/bundle-validation/spark_hadoop_mr/write.scala index 6c4745493e6cf..4d0065fa6e155 100644 --- a/packaging/bundle-validation/spark_hadoop_mr/write.scala +++ b/packaging/bundle-validation/spark_hadoop_mr/write.scala @@ -42,7 +42,6 @@ df.write.format("hudi"). option("hoodie.datasource.hive_sync.table", tableName). option("hoodie.datasource.hive_sync.partition_extractor_class", "org.apache.hudi.hive.SinglePartPartitionValueExtractor"). option("hoodie.datasource.hive_sync.mode", "hms"). - option("hoodie.datasource.hive_sync.metastore.uris", "thrift://localhost:9083/"). mode(Overwrite). save(basePath) diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml index d81c53be7f774..8dd07eabe7756 100644 --- a/packaging/hudi-aws-bundle/pom.xml +++ b/packaging/hudi-aws-bundle/pom.xml @@ -86,19 +86,6 @@ io.dropwizard.metrics:metrics-core com.beust:jcommander commons-io:commons-io - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.openjdk.jol:jol-core @@ -111,21 +98,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - com.amazonaws. org.apache.hudi.com.amazonaws. @@ -138,74 +110,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-datahub-sync-bundle/pom.xml b/packaging/hudi-datahub-sync-bundle/pom.xml index 1c52834088f1b..4b0b50c460104 100644 --- a/packaging/hudi-datahub-sync-bundle/pom.xml +++ b/packaging/hudi-datahub-sync-bundle/pom.xml @@ -75,19 +75,6 @@ io.acryl:datahub-client com.beust:jcommander commons-io:commons-io - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.apache.httpcomponents:httpasyncclient org.apache.httpcomponents:httpcore-nio org.openjdk.jol:jol-core @@ -98,93 +85,10 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index 9362a30aac324..16791012082d0 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -142,19 +142,6 @@ org.apache.thrift:libfb303 org.apache.orc:orc-core - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 commons-codec:commons-codec commons-io:commons-io org.openjdk.jol:jol-core @@ -173,21 +160,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - com.yammer.metrics. ${flink.bundle.shade.prefix}com.yammer.metrics. @@ -216,74 +188,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/packaging/hudi-gcp-bundle/pom.xml b/packaging/hudi-gcp-bundle/pom.xml index 40d20c62f58ac..231fb22b0ab6d 100644 --- a/packaging/hudi-gcp-bundle/pom.xml +++ b/packaging/hudi-gcp-bundle/pom.xml @@ -95,25 +95,12 @@ org.apache.hudi:hudi-common org.apache.hudi:hudi-hadoop-mr org.apache.hudi:hudi-sync-common + org.apache.hudi:hudi-hive-sync org.apache.hudi:hudi-gcp org.apache.parquet:parquet-avro - com.google.cloud:google-cloud-bigquery com.beust:jcommander commons-io:commons-io - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.openjdk.jol:jol-core @@ -122,93 +109,10 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false @@ -260,6 +164,12 @@ ${project.version} + + org.apache.hudi + hudi-hive-sync + ${project.version} + + org.apache.hudi hudi-gcp diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml index 646322d831980..4bf6b8d959e2b 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -70,19 +70,6 @@ org.apache.hudi:hudi-hadoop-mr org.apache.parquet:parquet-avro org.apache.avro:avro - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core commons-io:commons-io org.openjdk.jol:jol-core @@ -101,21 +88,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.apache.parquet.avro. org.apache.hudi.org.apache.parquet.avro. @@ -128,74 +100,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index 1b06cef863c34..38d1a97e84a29 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -73,19 +73,6 @@ com.beust:jcommander org.apache.avro:avro - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.apache.parquet:parquet-avro commons-io:commons-io org.openjdk.jol:jol-core @@ -96,93 +83,10 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 3f97914df75d1..340a54979b89f 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -87,19 +87,6 @@ org.apache.hudi:hudi-timeline-service org.apache.hudi:hudi-integ-test - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 commons-io:commons-io com.facebook.presto:presto-jdbc io.trino:trino-jdbc @@ -186,6 +173,7 @@ org.apache.spark.sql.avro. org.apache.hudi.org.apache.spark.sql.avro. + com.beust.jcommander. org.apache.hudi.com.beust.jcommander. @@ -206,21 +194,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.apache.hive.jdbc. org.apache.hudi.org.apache.hive.jdbc. @@ -293,73 +266,9 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - + org.eclipse.jetty. + org.apache.hudi.org.eclipse.jetty. @@ -401,12 +310,6 @@ - - io.javalin - javalin - 2.8.0 - - io.dropwizard.metrics metrics-core @@ -452,6 +355,13 @@ compile + + org.apache.hudi + hudi-timeline-service + ${project.version} + compile + + org.apache.hudi hudi-integ-test diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml index cb258f843c446..cc294acf10970 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -110,19 +110,6 @@ io.prometheus:simpleclient_common com.google.protobuf:protobuf-java - org.apache.hbase:hbase-client - org.apache.hbase:hbase-common - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.scala-lang:* commons-io:commons-io org.openjdk.jol:jol-core @@ -152,95 +139,10 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index bb9d56f1817e7..b64ebff0f8db8 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -73,19 +73,6 @@ org.apache.avro:avro org.codehaus.jackson:* org.apache.commons:commons-lang3 - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava commons-io:commons-io @@ -107,21 +94,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -150,74 +122,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index 694dea940cafc..9bb7cfeaa7b27 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -159,21 +159,7 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - + org.apache.hive.jdbc. ${spark.bundle.hive.shade.prefix}org.apache.hive.jdbc. @@ -222,75 +208,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index 9e48fdfb74c77..c6d2c30c78200 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -183,19 +183,6 @@ com.fasterxml.jackson.core:jackson-annotations com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 commons-io:commons-io log4j:log4j org.openjdk.jol:jol-core @@ -206,93 +193,10 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 907c444df9191..10e7da271b336 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -73,19 +73,6 @@ org.apache.parquet:parquet-avro org.apache.avro:avro org.codehaus.jackson:* - org.apache.hbase:hbase-common - org.apache.hbase:hbase-client - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.htrace:htrace-core4 com.yammer.metrics:metrics-core com.google.guava:guava commons-lang:commons-lang @@ -107,21 +94,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - com.yammer.metrics. org.apache.hudi.com.yammer.metrics. @@ -142,74 +114,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - false diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index b2fe32b1dff83..82602340d6715 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -146,19 +146,6 @@ org.apache.hive:hive-metastore org.apache.hive:hive-jdbc - org.apache.hbase:hbase-client - org.apache.hbase:hbase-common - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes @@ -191,21 +178,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.apache.hadoop.hive.metastore. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.metastore. @@ -246,74 +218,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index 9983f90623358..9641ccf61ebcf 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -129,19 +129,6 @@ org.apache.kafka:kafka_${scala.binary.version} com.101tec:zkclient org.apache.kafka:kafka-clients - org.apache.hbase:hbase-client - org.apache.hbase:hbase-common - org.apache.hbase:hbase-hadoop-compat - org.apache.hbase:hbase-hadoop2-compat - org.apache.hbase:hbase-metrics - org.apache.hbase:hbase-metrics-api - org.apache.hbase:hbase-protocol - org.apache.hbase:hbase-protocol-shaded - org.apache.hbase:hbase-server - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous - org.apache.hbase.thirdparty:hbase-shaded-netty - org.apache.hbase.thirdparty:hbase-shaded-protobuf - org.apache.htrace:htrace-core4 org.apache.curator:curator-framework org.apache.curator:curator-client org.apache.curator:curator-recipes @@ -170,21 +157,6 @@ org.apache.commons.io. org.apache.hudi.org.apache.commons.io. - - org.apache.hadoop.hbase. - org.apache.hudi.org.apache.hadoop.hbase. - - org.apache.hadoop.hbase.KeyValue$KeyComparator - - - - org.apache.hbase. - org.apache.hudi.org.apache.hbase. - - - org.apache.htrace. - org.apache.hudi.org.apache.htrace. - org.apache.hadoop.hive.metastore. ${utilities.bundle.hive.shade.prefix}org.apache.hadoop.hive.metastore. @@ -217,74 +189,6 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. - - - org.apache.hadoop.metrics2.MetricHistogram - org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram - - - - org.apache.hadoop.metrics2.MetricsExecutor - org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor - - - - org.apache.hadoop.metrics2.impl.JmxCacheBuster - org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster - - - org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper - - - - org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry - - - - org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl - - - - org.apache.hadoop.metrics2.lib.MutableFastCounter - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter - - - - org.apache.hadoop.metrics2.lib.MutableHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram - - - - org.apache.hadoop.metrics2.lib.MutableRangeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableSizeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram - - - - org.apache.hadoop.metrics2.lib.MutableTimeHistogram - org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram - - - - org.apache.hadoop.metrics2.util.MetricQuantile - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile - - - - org.apache.hadoop.metrics2.util.MetricSampleQuantiles - org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles - - diff --git a/pom.xml b/pom.xml index d2b67d8cc9fbe..664942e8297ad 100644 --- a/pom.xml +++ b/pom.xml @@ -270,7 +270,8 @@ - true + + false UTF-8 style/checkstyle.xml style/checkstyle-suppressions.xml @@ -438,6 +439,20 @@ org.apache.httpcomponents:httpclient org.apache.httpcomponents:httpcore org.apache.httpcomponents:fluent-hc + + org.apache.hbase:hbase-client + org.apache.hbase:hbase-common + org.apache.hbase:hbase-hadoop-compat + org.apache.hbase:hbase-hadoop2-compat + org.apache.hbase:hbase-metrics + org.apache.hbase:hbase-metrics-api + org.apache.hbase:hbase-protocol + org.apache.hbase:hbase-protocol-shaded + org.apache.hbase:hbase-server + org.apache.hbase.thirdparty:hbase-shaded-miscellaneous + org.apache.hbase.thirdparty:hbase-shaded-netty + org.apache.hbase.thirdparty:hbase-shaded-protobuf + org.apache.htrace:htrace-core4 @@ -463,6 +478,93 @@ org.apache.http. org.apache.hudi.org.apache.http. + + + org.apache.hadoop.hbase. + org.apache.hudi.org.apache.hadoop.hbase. + + org.apache.hadoop.hbase.KeyValue$KeyComparator + + + + org.apache.hbase. + org.apache.hudi.org.apache.hbase. + + + org.apache.htrace. + org.apache.hudi.org.apache.htrace. + + + + org.apache.hadoop.metrics2.MetricHistogram + org.apache.hudi.org.apache.hadoop.metrics2.MetricHistogram + + + + org.apache.hadoop.metrics2.MetricsExecutor + org.apache.hudi.org.apache.hadoop.metrics2.MetricsExecutor + + + + org.apache.hadoop.metrics2.impl.JmxCacheBuster + org.apache.hudi.org.apache.hadoop.metrics2.impl.JmxCacheBuster + + + + org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + org.apache.hudi.org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper + + + + org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + org.apache.hudi.org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry + + + + org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + org.apache.hudi.org.apache.hadoop.metrics2.lib.MetricsExecutorImpl + + + + org.apache.hadoop.metrics2.lib.MutableFastCounter + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableFastCounter + + + + org.apache.hadoop.metrics2.lib.MutableHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableHistogram + + + + org.apache.hadoop.metrics2.lib.MutableRangeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableRangeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableSizeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableSizeHistogram + + + + org.apache.hadoop.metrics2.lib.MutableTimeHistogram + org.apache.hudi.org.apache.hadoop.metrics2.lib.MutableTimeHistogram + + + + org.apache.hadoop.metrics2.util.MetricQuantile + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricQuantile + + + + org.apache.hadoop.metrics2.util.MetricSampleQuantiles + org.apache.hudi.org.apache.hadoop.metrics2.util.MetricSampleQuantiles + + @@ -487,7 +589,7 @@ org.scalatest scalatest-maven-plugin - 1.0 + 2.2.0 ${skipUTs} ${project.build.directory}/surefire-reports @@ -2153,6 +2255,12 @@ flink1.15 + + ${flink1.15.version} + hudi-flink1.15.x + 1.15 + 1.5.6 + flink1.15 diff --git a/scripts/release/cut_release_branch.sh b/scripts/release/cut_release_branch.sh index 49b58391bddd7..44ed5d7742b79 100755 --- a/scripts/release/cut_release_branch.sh +++ b/scripts/release/cut_release_branch.sh @@ -56,12 +56,13 @@ else fi if [[ -z "$RELEASE" || -z "$NEXT_VERSION_IN_BASE_BRANCH" || -z "$RC_NUM" ]]; then - echo "This sricpt needs to be ran with params, please run with -h to get more instructions." + echo "This script needs to be ran with params, please run with -h to get more instructions." exit fi MASTER_BRANCH=master +NEXT_VERSION_BRANCH=MINOR-move-to-${NEXT_VERSION_IN_BASE_BRANCH} RELEASE_BRANCH=release-${RELEASE} GITHUB_REPO_URL=git@github.com:apache/hudi.git HUDI_ROOT_DIR=hudi @@ -71,6 +72,7 @@ echo "=====================Environment Variables=====================" echo "version: ${RELEASE}" echo "next_release: ${NEXT_VERSION_IN_BASE_BRANCH}" echo "working master branch: ${MASTER_BRANCH}" +echo "working next-version branch: ${NEXT_VERSION_BRANCH}" echo "working release branch: ${RELEASE_BRANCH}" echo "local repo dir: ~/${LOCAL_CLONE_DIR}/${HUDI_ROOT_DIR}" echo "RC_NUM: $RC_NUM" @@ -90,15 +92,16 @@ cd ${HUDI_ROOT_DIR} git branch ${RELEASE_BRANCH} git checkout ${MASTER_BRANCH} +git checkout -b ${NEXT_VERSION_BRANCH} echo "====================Current working branch=====================" -echo ${MASTER_BRANCH} +echo ${NEXT_VERSION_BRANCH} echo "===============================================================" # Update master branch mvn versions:set -DnewVersion=${NEXT_VERSION_IN_BASE_BRANCH}-SNAPSHOT -echo "==============Update master branch as following================" +echo "===========Update next-version branch as following=============" git diff echo "===============================================================" @@ -110,14 +113,11 @@ if [[ $confirmation != "y" ]]; then exit fi -git commit -am "Moving to ${NEXT_VERSION_IN_BASE_BRANCH}-SNAPSHOT on master branch." +git commit -am "[MINOR] Moving to ${NEXT_VERSION_IN_BASE_BRANCH}-SNAPSHOT on master branch." -if git push origin ${MASTER_BRANCH}; then - break -else - clean_up - exit -fi +echo "===============================================================" +echo "!!Please open a PR based on ${NEXT_VERSION_BRANCH} branch for approval!! [Press ENTER to continue]" +read confirmation # Checkout and update release branch git checkout ${RELEASE_BRANCH} diff --git a/scripts/release/deploy_staging_jars.sh b/scripts/release/deploy_staging_jars.sh index 15497ad3cf009..3d6710ff5923d 100755 --- a/scripts/release/deploy_staging_jars.sh +++ b/scripts/release/deploy_staging_jars.sh @@ -61,8 +61,12 @@ declare -a ALL_VERSION_OPTS=( "-Dscala-2.12 -Dspark3 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark3-bundle_2.12 # utilities bundles (legacy) (overwriting previous uploads) -"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-utilities-bundle -am" # utilities-bundle_2.11 is for spark 2.4 only -"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-utilities-bundle -am" # utilities-bundle_2.12 is for spark 3.1 only +"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-utilities-bundle -am" # hudi-utilities-bundle_2.11 is for spark 2.4 only +"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-utilities-bundle -am" # hudi-utilities-bundle_2.12 is for spark 3.1 only + +# utilities slim bundles +"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-utilities-slim-bundle -am" # hudi-utilities-slim-bundle_2.11 +"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-utilities-slim-bundle -am" # hudi-utilities-slim-bundle_2.12 # flink bundles (overwriting previous uploads) "-Dscala-2.12 -Dflink1.13 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" @@ -97,17 +101,26 @@ elif [ "$#" == "1" ]; then exit 1 fi +COMMON_OPTIONS="-DdeployArtifacts=true -DskipTests -DretryFailedDeploymentCount=10" + for v in "${ALL_VERSION_OPTS[@]}" do - # clean everything before any round of depoyment - #$MVN clean - #echo "Building with options ${v}" - #echo "install Command: $MVN install "$COMMON_OPTIONS" "${v}"" - #$MVN install "${v}" "$COMMON_OPTIONS" - #echo "Deploying to repository.apache.org with version options $COMMON_OPTIONS ${v%-am}" - #echo "Command execute: $MVN clean install deploy "${v%-am}" "$COMMON_OPTIONS"" + # TODO: consider cleaning all modules by listing directories instead of specifying profile + if [[ "$v" == *"$BUNDLE_MODULES_EXCLUDED" ]]; then + # When deploying jars with bundle exclusions, we still need to build the bundles, + # by removing "-pl -packaging/hudi-aws-bundle...", otherwise the build fails. + v1=${v%${BUNDLE_MODULES_EXCLUDED}} + echo "Cleaning everything before any deployment" + $MVN clean $COMMON_OPTIONS ${v1%-pl } + echo "Building with options ${v1%-pl }" + $MVN install $COMMON_OPTIONS ${v1%-pl } + else + echo "Cleaning everything before any deployment" + $MVN clean $COMMON_OPTIONS ${v} + echo "Building with options ${v}" + $MVN install $COMMON_OPTIONS ${v} + fi + echo "Deploying to repository.apache.org with version options ${v%-am}" # remove `-am` option to only deploy intended modules - #$MVN clean deploy "$COMMON_OPTIONS" "${v%-am}" - COMMON_OPTIONS="${v} -DdeployArtifacts=true -DskipTests -DretryFailedDeploymentCount=10" - $MVN clean deploy $COMMON_OPTIONS + $MVN deploy $COMMON_OPTIONS ${v%-am} done diff --git a/scripts/release/validate_staged_bundles.sh b/scripts/release/validate_staged_bundles.sh index cd0c4b101d816..9df615c8141b1 100755 --- a/scripts/release/validate_staged_bundles.sh +++ b/scripts/release/validate_staged_bundles.sh @@ -29,7 +29,7 @@ VERSION=$2 STAGING_REPO="https://repository.apache.org/content/repositories/${REPO}/org/apache/hudi" declare -a BUNDLE_URLS=( - +"${STAGING_REPO}/hudi-aws-bundle/${VERSION}/hudi-aws-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-datahub-sync-bundle/${VERSION}/hudi-datahub-sync-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-flink1.13-bundle/${VERSION}/hudi-flink1.13-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-flink1.14-bundle/${VERSION}/hudi-flink1.14-bundle-${VERSION}.jar" @@ -39,6 +39,7 @@ declare -a BUNDLE_URLS=( "${STAGING_REPO}/hudi-hive-sync-bundle/${VERSION}/hudi-hive-sync-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-integ-test-bundle/${VERSION}/hudi-integ-test-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-kafka-connect-bundle/${VERSION}/hudi-kafka-connect-bundle-${VERSION}.jar" +"${STAGING_REPO}/hudi-metaserver-server-bundle/${VERSION}/hudi-metaserver-server-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-presto-bundle/${VERSION}/hudi-presto-bundle-${VERSION}.jar" "${STAGING_REPO}/hudi-spark-bundle_2.11/${VERSION}/hudi-spark-bundle_2.11-${VERSION}.jar" "${STAGING_REPO}/hudi-spark-bundle_2.12/${VERSION}/hudi-spark-bundle_2.12-${VERSION}.jar" diff --git a/scripts/release/validate_staged_release.sh b/scripts/release/validate_staged_release.sh index 879583a45d9da..641047ee9660a 100755 --- a/scripts/release/validate_staged_release.sh +++ b/scripts/release/validate_staged_release.sh @@ -126,11 +126,11 @@ cd hudi-${ARTIFACT_SUFFIX} ### BEGIN: Binary Files Check echo "Checking for binary files in source release" -numBinaryFiles=`find . -iname '*' | xargs -I {} file -I {} | grep -va directory | grep -v "/src/test/" | grep -va 'application/json' | grep -va 'text/' | grep -va 'application/xml' | grep -va 'application/json' | wc -l | sed -e s'/ //g'` +numBinaryFiles=`find . -iname '*' | xargs -I {} file -I {} | grep -va directory | grep -v "release/" | grep -v "/src/test/" | grep -va 'application/json' | grep -va 'text/' | grep -va 'application/xml' | grep -va 'application/json' | wc -l | sed -e s'/ //g'` if [ "$numBinaryFiles" -gt "0" ]; then echo -e "There were non-text files in source release. [ERROR]\n Please check below\n" - find . -iname '*' | xargs -I {} file -I {} | grep -va directory | grep -v "/src/test/" | grep -va 'application/json' | grep -va 'text/' | grep -va 'application/xml' + find . -iname '*' | xargs -I {} file -I {} | grep -va directory | grep -v "release/release_guide" | grep -v "/src/test/" | grep -va 'application/json' | grep -va 'text/' | grep -va 'application/xml' exit 1 fi echo -e "\t\tNo Binary Files in Source Release? - [OK]\n" @@ -163,10 +163,10 @@ echo -e "\t\tNotice file exists ? [OK]\n" ### Licensing Check echo "Performing custom Licensing Check " -numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'banner.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" | wc -l` +numfilesWithNoLicense=`find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v KEYS | grep -v '.mailmap' | grep -v '.sqltemplate' | grep -v 'banner.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" | wc -l` if [ "$numfilesWithNoLicense" -gt "0" ]; then echo "There were some source files that did not have Apache License [ERROR]" - find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" + find . -iname '*' -type f | grep -v NOTICE | grep -v LICENSE | grep -v '.jpg' | grep -v '.json' | grep -v '.hfile' | grep -v '.data' | grep -v '.commit' | grep -v DISCLAIMER | grep -v '.sqltemplate' | grep -v KEYS | grep -v '.mailmap' | grep -v 'banner.txt' | grep -v "fixtures" | xargs grep -L "Licensed to the Apache Software Foundation (ASF)" exit 1 fi echo -e "\t\tLicensing Check Passed [OK]\n"