diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 065c4b7a5d53d..9c8eb5c8448d8 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -15,13 +15,37 @@ on: - '**.pdf' - '**.png' - '**.svg' + - '**.yaml' + - '**.yml' + - '.gitignore' branches: - master - 'release-*' env: - MVN_ARGS: -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn + MVN_ARGS: -e -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn + SPARK_COMMON_MODULES: hudi-spark-datasource/hudi-spark,hudi-spark-datasource/hudi-spark-common jobs: + validate-source: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 8 + uses: actions/setup-java@v2 + with: + java-version: '8' + distribution: 'adopt' + architecture: x64 + - name: Check Binary Files + run: ./scripts/release/validate_source_binary_files.sh + - name: Check Copyright + run: | + ./scripts/release/create_source_directory.sh hudi-tmp-repo + cd hudi-tmp-repo + ./scripts/release/validate_source_copyright.sh + - name: RAT check + run: ./scripts/release/validate_source_rat.sh + test-spark: runs-on: ubuntu-latest strategy: @@ -29,18 +53,23 @@ jobs: include: - scalaProfile: "scala-2.11" sparkProfile: "spark2.4" + sparkModules: "hudi-spark-datasource/hudi-spark2" - scalaProfile: "scala-2.12" sparkProfile: "spark2.4" + sparkModules: "hudi-spark-datasource/hudi-spark2" - scalaProfile: "scala-2.12" sparkProfile: "spark3.1" + sparkModules: "hudi-spark-datasource/hudi-spark3.1.x" - scalaProfile: "scala-2.12" sparkProfile: "spark3.2" + sparkModules: "hudi-spark-datasource/hudi-spark3.2.x" - scalaProfile: "scala-2.12" sparkProfile: "spark3.3" + sparkModules: "hudi-spark-datasource/hudi-spark3.3.x" steps: - uses: actions/checkout@v2 @@ -66,9 +95,18 @@ jobs: env: SCALA_PROFILE: ${{ matrix.scalaProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} if: ${{ !endsWith(env.SPARK_PROFILE, '2.4') }} # skip test spark 2.4 as it's covered by Azure CI run: - mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl hudi-common,hudi-spark-datasource/hudi-spark $MVN_ARGS + mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "hudi-common,$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS + - name: FT - Spark + env: + SCALA_PROFILE: ${{ matrix.scalaProfile }} + SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_MODULES: ${{ matrix.sparkModules }} + if: ${{ !endsWith(env.SPARK_PROFILE, '2.4') }} # skip test spark 2.4 as it's covered by Azure CI + run: + mvn test -Pfunctional-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -pl "$SPARK_COMMON_MODULES,$SPARK_MODULES" $MVN_ARGS test-flink: runs-on: ubuntu-latest @@ -107,12 +145,16 @@ jobs: include: - flinkProfile: 'flink1.16' sparkProfile: 'spark3.3' + sparkRuntime: 'spark3.3.2' - flinkProfile: 'flink1.15' sparkProfile: 'spark3.3' + sparkRuntime: 'spark3.3.1' - flinkProfile: 'flink1.14' sparkProfile: 'spark3.2' + sparkRuntime: 'spark3.2.3' - flinkProfile: 'flink1.13' sparkProfile: 'spark3.1' + sparkRuntime: 'spark3.1.3' steps: - uses: actions/checkout@v2 - name: Set up JDK 8 @@ -127,13 +169,33 @@ jobs: SPARK_PROFILE: ${{ matrix.sparkProfile }} SCALA_PROFILE: 'scala-2.12' run: | - mvn clean package -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -Pintegration-tests -DskipTests=true -Phudi-platform-service $MVN_ARGS - mvn clean package -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -pl packaging/hudi-flink-bundle -am -Davro.version=1.10.0 -DskipTests=true $MVN_ARGS - - name: IT - Bundle Validation + mvn clean package -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -DdeployArtifacts=true -DskipTests=true $MVN_ARGS + # TODO remove the sudo below. It's a needed workaround as detailed in HUDI-5708. + sudo chown -R "$USER:$(id -g -n)" hudi-platform-service/hudi-metaserver/target/generated-sources + mvn clean package -D"$SCALA_PROFILE" -D"$FLINK_PROFILE" -DdeployArtifacts=true -DskipTests=true $MVN_ARGS -pl packaging/hudi-flink-bundle -am -Davro.version=1.10.0 + - name: IT - Bundle Validation - OpenJDK 8 + env: + FLINK_PROFILE: ${{ matrix.flinkProfile }} + SPARK_RUNTIME: ${{ matrix.sparkRuntime }} + SCALA_PROFILE: 'scala-2.12' + run: | + HUDI_VERSION=$(mvn help:evaluate -Dexpression=project.version -q -DforceStdout) + ./packaging/bundle-validation/ci_run.sh $HUDI_VERSION openjdk8 + - name: IT - Bundle Validation - OpenJDK 11 + env: + FLINK_PROFILE: ${{ matrix.flinkProfile }} + SPARK_RUNTIME: ${{ matrix.sparkRuntime }} + SCALA_PROFILE: 'scala-2.12' + run: | + HUDI_VERSION=$(mvn help:evaluate -Dexpression=project.version -q -DforceStdout) + ./packaging/bundle-validation/ci_run.sh $HUDI_VERSION openjdk11 + - name: IT - Bundle Validation - OpenJDK 17 env: FLINK_PROFILE: ${{ matrix.flinkProfile }} SPARK_PROFILE: ${{ matrix.sparkProfile }} + SPARK_RUNTIME: ${{ matrix.sparkRuntime }} SCALA_PROFILE: 'scala-2.12' + if: ${{ endsWith(env.SPARK_PROFILE, '3.3') }} # Only Spark 3.3 supports Java 17 as of now run: | HUDI_VERSION=$(mvn help:evaluate -Dexpression=project.version -q -DforceStdout) - ./packaging/bundle-validation/ci_run.sh $HUDI_VERSION + ./packaging/bundle-validation/ci_run.sh $HUDI_VERSION openjdk17 diff --git a/azure-pipelines.yml b/azure-pipelines-20230430.yml similarity index 97% rename from azure-pipelines.yml rename to azure-pipelines-20230430.yml index 73d7bdecd7cbb..fe13e36235fb0 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines-20230430.yml @@ -13,6 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +# NOTE: +# This config file defines how Azure CI runs tests with Spark 2.4 and Flink 1.17 profiles. +# PRs will need to keep in sync with master's version to trigger the CI runs. + trigger: branches: include: @@ -85,7 +89,7 @@ parameters: variables: BUILD_PROFILES: '-Dscala-2.11 -Dspark2.4 -Dflink1.16' PLUGIN_OPTS: '-Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -ntp -B -V -Pwarn-log -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.shade=warn -Dorg.slf4j.simpleLogger.log.org.apache.maven.plugins.dependency=warn' - MVN_OPTS_INSTALL: '-DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS)' + MVN_OPTS_INSTALL: '-Phudi-platform-service -DskipTests $(BUILD_PROFILES) $(PLUGIN_OPTS)' MVN_OPTS_TEST: '-fae -Pwarn-log $(BUILD_PROFILES) $(PLUGIN_OPTS)' SPARK_VERSION: '2.4.4' HADOOP_VERSION: '2.7' diff --git a/doap_HUDI.rdf b/doap_HUDI.rdf index 0817e38f99318..761f35c0808c0 100644 --- a/doap_HUDI.rdf +++ b/doap_HUDI.rdf @@ -111,6 +111,16 @@ 2022-12-28 0.12.2 + + Apache Hudi 0.13.0 + 2023-02-25 + 0.13.0 + + + Apache Hudi 0.12.3 + 2023-04-23 + 0.12.3 + 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/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml index 2e4e75ef5a20b..3a4b65c39e545 100644 --- a/docker/hoodie/hadoop/base/pom.xml +++ b/docker/hoodie/hadoop/base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/base_java11/pom.xml b/docker/hoodie/hadoop/base_java11/pom.xml index 9871ccd264f85..d6ddbd1b16c9e 100644 --- a/docker/hoodie/hadoop/base_java11/pom.xml +++ b/docker/hoodie/hadoop/base_java11/pom.xml @@ -20,7 +20,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml index 1a22fddc52db1..afe52304a35b6 100644 --- a/docker/hoodie/hadoop/datanode/pom.xml +++ b/docker/hoodie/hadoop/datanode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml index dfdf3ab9a664f..ae9f56fdd8bca 100644 --- a/docker/hoodie/hadoop/historyserver/pom.xml +++ b/docker/hoodie/hadoop/historyserver/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml index 033b15dd90b4a..def902dca118b 100644 --- a/docker/hoodie/hadoop/hive_base/pom.xml +++ b/docker/hoodie/hadoop/hive_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml index 4ea1ad42fd619..5a8caea403a1e 100644 --- a/docker/hoodie/hadoop/namenode/pom.xml +++ b/docker/hoodie/hadoop/namenode/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml index 80fac38d01799..63985fe2ed6f2 100644 --- a/docker/hoodie/hadoop/pom.xml +++ b/docker/hoodie/hadoop/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../../pom.xml 4.0.0 diff --git a/docker/hoodie/hadoop/prestobase/pom.xml b/docker/hoodie/hadoop/prestobase/pom.xml index b0bbeb663870d..f546cb460692c 100644 --- a/docker/hoodie/hadoop/prestobase/pom.xml +++ b/docker/hoodie/hadoop/prestobase/pom.xml @@ -20,7 +20,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml index 6e79a3952bbf0..a3c38da2a5571 100644 --- a/docker/hoodie/hadoop/spark_base/pom.xml +++ b/docker/hoodie/hadoop/spark_base/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml index 65358506a11ae..d25f6aeb67677 100644 --- a/docker/hoodie/hadoop/sparkadhoc/pom.xml +++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml index cdaf644231974..91aa3d3c06930 100644 --- a/docker/hoodie/hadoop/sparkmaster/pom.xml +++ b/docker/hoodie/hadoop/sparkmaster/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml index 68486ada6845c..c9e9fce22c715 100644 --- a/docker/hoodie/hadoop/sparkworker/pom.xml +++ b/docker/hoodie/hadoop/sparkworker/pom.xml @@ -19,7 +19,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinobase/pom.xml b/docker/hoodie/hadoop/trinobase/pom.xml index dea2efbab5ecf..fe5f2823a81a4 100644 --- a/docker/hoodie/hadoop/trinobase/pom.xml +++ b/docker/hoodie/hadoop/trinobase/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinocoordinator/pom.xml b/docker/hoodie/hadoop/trinocoordinator/pom.xml index 5b162e7f6d443..60e9c7e5240a5 100644 --- a/docker/hoodie/hadoop/trinocoordinator/pom.xml +++ b/docker/hoodie/hadoop/trinocoordinator/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/docker/hoodie/hadoop/trinoworker/pom.xml b/docker/hoodie/hadoop/trinoworker/pom.xml index f9a2197339eec..2e0d84d444c35 100644 --- a/docker/hoodie/hadoop/trinoworker/pom.xml +++ b/docker/hoodie/hadoop/trinoworker/pom.xml @@ -22,7 +22,7 @@ hudi-hadoop-docker org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 pom diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml index dd484561f8a49..105670dfb3420 100644 --- a/hudi-aws/pom.xml +++ b/hudi-aws/pom.xml @@ -19,12 +19,12 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-aws - 0.13.0-SNAPSHOT + 0.13.1 hudi-aws jar 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..eed9486d69cd0 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,11 @@ 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(); + try (AwsGlueCatalogSyncTool tool = new AwsGlueCatalogSyncTool(props, hadoopConf)) { + tool.syncHoodieTable(); + } } } diff --git a/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java index 1894b8641c1be..15e81bc90e394 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java +++ b/hudi-aws/src/main/java/org/apache/hudi/config/DynamoDbBasedLockConfig.java @@ -34,10 +34,11 @@ * Hoodie Configs for Locks. */ @ConfigClassProperty(name = "DynamoDB based Locks Configurations", - groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Configs that control DynamoDB based locking mechanisms required for concurrency control " - + " between writers to a Hudi table. Concurrency between Hudi's own table services " - + " are auto managed internally.") + groupName = ConfigGroups.Names.WRITE_CLIENT, + subGroupName = ConfigGroups.SubGroupNames.LOCK, + description = "Configs that control DynamoDB based locking mechanisms required for concurrency control " + + " between writers to a Hudi table. Concurrency between Hudi's own table services " + + " are auto managed internally.") public class DynamoDbBasedLockConfig extends HoodieConfig { // configs for DynamoDb based locks @@ -81,7 +82,7 @@ public class DynamoDbBasedLockConfig extends HoodieConfig { .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "billing_mode") .defaultValue(BillingMode.PAY_PER_REQUEST.name()) .sinceVersion("0.10.0") - .withDocumentation("For DynamoDB based lock provider, by default it is PAY_PER_REQUEST mode"); + .withDocumentation("For DynamoDB based lock provider, by default it is `PAY_PER_REQUEST` mode. Alternative is `PROVISIONED`."); public static final ConfigProperty DYNAMODB_LOCK_READ_CAPACITY = ConfigProperty .key(DYNAMODB_BASED_LOCK_PROPERTY_PREFIX + "read_capacity") diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml index f6f11433af6de..577f38a0d2565 100644 --- a/hudi-cli/pom.xml +++ b/hudi-cli/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java index 7dd889e58e374..29a8ccac383e6 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java @@ -44,6 +44,8 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.exception.HoodieException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.springframework.shell.standard.ShellComponent; import org.springframework.shell.standard.ShellMethod; import org.springframework.shell.standard.ShellOption; @@ -67,6 +69,8 @@ @ShellComponent public class ExportCommand { + private static final Logger LOG = LoggerFactory.getLogger(ExportCommand.class); + @ShellMethod(key = "export instants", value = "Export Instants and their metadata from the Timeline") public String exportInstants( @ShellOption(value = {"--limit"}, help = "Limit Instants", defaultValue = "-1") final Integer limit, @@ -77,7 +81,7 @@ public String exportInstants( throws Exception { final String basePath = HoodieCLI.getTableMetaClient().getBasePath(); - final Path archivePath = new Path(basePath + "/.hoodie/.commits_.archive*"); + final Path archivePath = new Path(HoodieCLI.getTableMetaClient().getArchivePath()); final Set actionSet = new HashSet(Arrays.asList(filter.split(","))); int numExports = limit == -1 ? Integer.MAX_VALUE : limit; int numCopied = 0; @@ -121,7 +125,7 @@ private int copyArchivedInstants(List statuses, Set actionSe Reader reader = HoodieLogFormat.newReader(fileSystem, new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); // read the avro blocks - while (reader.hasNext() && copyCount < limit) { + while (reader.hasNext() && copyCount++ < limit) { HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); try (ClosableIterator> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) { while (recordItr.hasNext()) { @@ -158,11 +162,12 @@ private int copyArchivedInstants(List statuses, Set actionSe } final String instantTime = archiveEntryRecord.get("commitTime").toString(); + if (metadata == null) { + LOG.error("Could not load metadata for action " + action + " at instant time " + instantTime); + continue; + } final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action; writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true)); - if (++copyCount == limit) { - break; - } } } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index 9a0e485fc9fa0..075b809e05c0d 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -222,7 +222,7 @@ public String showLogFileRecords( .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) + .withOptimizedLogBlocksScan(Boolean.parseBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.defaultValue())) .build(); for (HoodieRecord hoodieRecord : scanner) { Option record = hoodieRecord.toIndexedRecord(readerSchema, new Properties()); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java index 4b817473f05d8..8f05d523b8c8d 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java @@ -159,7 +159,9 @@ public String overwriteHoodieProperties( HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); Properties newProps = new Properties(); - newProps.load(new FileInputStream(overwriteFilePath)); + try (FileInputStream fileInputStream = new FileInputStream(overwriteFilePath)) { + newProps.load(fileInputStream); + } Map oldProps = client.getTableConfig().propsMap(); Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); HoodieTableConfig.create(client.getFs(), metaPathDir, newProps); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java index b3dfaf5ab73bc..0479a5c20bf29 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java @@ -200,7 +200,9 @@ public String updateTableConfig( Map oldProps = client.getTableConfig().propsMap(); Properties updatedProps = new Properties(); - updatedProps.load(new FileInputStream(updatePropsFilePath)); + try (FileInputStream fileInputStream = new FileInputStream(updatePropsFilePath)) { + updatedProps.load(fileInputStream); + } Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME); HoodieTableConfig.update(client.getFs(), metaPathDir, updatedProps); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java index 1cffdf941f917..fd09a27271a85 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java @@ -57,10 +57,18 @@ public static SparkLauncher initLauncher(String propertiesFile) throws URISyntax } File libDirectory = new File(new File(currentJar).getParent(), "lib"); - // This lib directory may be not required, such as providing libraries through a bundle jar if (libDirectory.exists()) { + // When directly using hudi-cli module, the jars under the lib directory + // generated by the compilation is required Arrays.stream(libDirectory.list()).forEach(library -> - sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath())); + sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath())); + } else { + // When using hudi-cli-bundle, we also need to add the hudi-spark*-bundle + // so that the Hudi Spark job can be launched + String sparkBundleJarPath = System.getenv("SPARK_BUNDLE_JAR"); + if (!StringUtils.isNullOrEmpty(sparkBundleJarPath)) { + sparkLauncher.addJar(sparkBundleJarPath); + } } return sparkLauncher; } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java index aff12422f6ad4..261002c9327d8 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java @@ -233,7 +233,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) + .withOptimizedLogBlocksScan(Boolean.parseBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.defaultValue())) .build(); Iterator records = scanner.iterator(); diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml index 82dee31157362..b00d8e2cea78a 100644 --- a/hudi-client/hudi-client-common/pom.xml +++ b/hudi-client/hudi-client-common/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-client-common - 0.13.0-SNAPSHOT + 0.13.1 hudi-client-common jar @@ -258,27 +258,6 @@ org.jacoco jacoco-maven-plugin - - net.alchim31.maven - scala-maven-plugin - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - org.apache.maven.plugins maven-compiler-plugin diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java index a5faaffe3d7ad..b8409c7d19b4e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java @@ -170,7 +170,8 @@ public HoodieHeartbeatClient getHeartbeatClient() { * * @param table A hoodie table instance created after transaction starts so that the latest commits and files are captured. * @param metadata Current committing instant's metadata - * @param pendingInflightAndRequestedInstants + * @param pendingInflightAndRequestedInstants Pending instants on the timeline + * * @see {@link BaseHoodieWriteClient#preCommit} * @see {@link BaseHoodieTableServiceClient#preCommit} */ diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 390bc4b97148a..5245c4bba2145 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPlan; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -48,6 +49,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieRollbackException; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -71,6 +73,7 @@ import java.util.stream.Stream; import static org.apache.hudi.common.util.ValidationUtils.checkArgument; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit; public abstract class BaseHoodieTableServiceClient extends BaseHoodieClient implements RunsTableService { @@ -85,8 +88,10 @@ public abstract class BaseHoodieTableServiceClient extends BaseHoodieClient i protected Set pendingInflightAndRequestedInstants; - protected BaseHoodieTableServiceClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig, Option.empty()); + protected BaseHoodieTableServiceClient(HoodieEngineContext context, + HoodieWriteConfig clientConfig, + Option timelineService) { + super(context, clientConfig, timelineService); } protected void startAsyncCleanerService(BaseHoodieWriteClient writeClient) { @@ -659,8 +664,41 @@ protected Map> getPendingRollbackInfos return infoMap; } + /** + * Rolls back the failed delta commits corresponding to the indexing action. + * Such delta commits are identified based on the suffix `METADATA_INDEXER_TIME_SUFFIX` ("004"). + *

+ * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks + * in the metadata table is landed. + * + * @return {@code true} if rollback happens; {@code false} otherwise. + */ + protected boolean rollbackFailedIndexingCommits() { + HoodieTable table = createTable(config, hadoopConf); + List instantsToRollback = getFailedIndexingCommitsToRollback(table.getMetaClient()); + Map> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient()); + instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); + rollbackFailedWrites(pendingRollbacks); + return !pendingRollbacks.isEmpty(); + } + + protected List getFailedIndexingCommitsToRollback(HoodieTableMetaClient metaClient) { + Stream inflightInstantsStream = metaClient.getCommitsTimeline() + .filter(instant -> !instant.isCompleted() + && isIndexingCommit(instant.getTimestamp())) + .getInstantsAsStream(); + return inflightInstantsStream.filter(instant -> { + try { + return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); + } catch (IOException io) { + throw new HoodieException("Failed to check heartbeat for instant " + instant, io); + } + }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + } + /** * Rollback all failed writes. + * * @return true if rollback was triggered. false otherwise. */ protected Boolean rollbackFailedWrites() { @@ -699,6 +737,19 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H Stream inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(metaClient) .getReverseOrderedInstants(); if (cleaningPolicy.isEager()) { + // Metadata table uses eager cleaning policy, but we need to exclude inflight delta commits + // from the async indexer (`HoodieIndexer`). + // TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks in the + // metadata table is landed. + if (HoodieTableMetadata.isMetadataTable(metaClient.getBasePathV2().toString())) { + return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> { + if (curInstantTime.isPresent()) { + return !entry.equals(curInstantTime.get()); + } else { + return !isIndexingCommit(entry); + } + }).collect(Collectors.toList()); + } return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> { if (curInstantTime.isPresent()) { return !entry.equals(curInstantTime.get()); 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 c3260914bd51b..3fe0fdd330f0b 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 @@ -491,7 +491,7 @@ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instan public void preWrite(String instantTime, WriteOperationType writeOperationType, HoodieTableMetaClient metaClient) { setOperationType(writeOperationType); - this.lastCompletedTxnAndMetadata = txnManager.isOptimisticConcurrencyControlEnabled() + this.lastCompletedTxnAndMetadata = txnManager.isLockRequired() ? TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient) : Option.empty(); this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(metaClient); this.pendingInflightAndRequestedInstants.remove(instantTime); @@ -519,6 +519,8 @@ public void preWrite(String instantTime, WriteOperationType writeOperationType, */ protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { + context.setJobStatus(this.getClass().getSimpleName(),"Cleaning up marker directories for commit " + instantTime + " in table " + + config.getTableName()); // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); @@ -533,7 +535,8 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, Stri */ protected void mayBeCleanAndArchive(HoodieTable table) { autoCleanOnCommit(); - autoArchiveOnCommit(table); + // reload table to that timeline reflects the clean commit + autoArchiveOnCommit(createTable(config, hadoopConf)); } protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { @@ -1074,7 +1077,8 @@ protected boolean scheduleCleaningAtInstant(String instantTime, Option scheduleTableService(Option> extraMetadata, TableServiceType tableServiceType) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); @@ -1086,20 +1090,11 @@ public Option scheduleTableService(Option> extraMeta * * @param extraMetadata Metadata to pass onto the scheduled service instant * @param tableServiceType Type of table service to schedule - * @return + * + * @return The given instant time option or empty if no table service plan is scheduled */ - public Option scheduleTableService(String instantTime, Option> extraMetadata, - TableServiceType tableServiceType) { - // A lock is required to guard against race conditions between an on-going writer and scheduling a table service. - final Option inflightInstant = Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, - tableServiceType.getAction(), instantTime)); - try { - this.txnManager.beginTransaction(inflightInstant, Option.empty()); - LOG.info("Scheduling table service " + tableServiceType); - return tableServiceClient.scheduleTableServiceInternal(instantTime, extraMetadata, tableServiceType); - } finally { - this.txnManager.endTransaction(inflightInstant); - } + public Option scheduleTableService(String instantTime, Option> extraMetadata, TableServiceType tableServiceType) { + return tableServiceClient.scheduleTableService(instantTime, extraMetadata, tableServiceType); } public HoodieMetrics getMetrics() { @@ -1277,8 +1272,21 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta } } + /** + * Rolls back the failed delta commits corresponding to the indexing action. + *

+ * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks + * in the metadata table is landed. + * + * @return {@code true} if rollback happens; {@code false} otherwise. + */ + public boolean lazyRollbackFailedIndexing() { + return tableServiceClient.rollbackFailedIndexingCommits(); + } + /** * Rollback failed writes if any. + * * @return true if rollback happened. false otherwise. */ public boolean rollbackFailedWrites() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/FailOnFirstErrorWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/FailOnFirstErrorWriteStatus.java new file mode 100644 index 0000000000000..40ccae0f000f1 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/FailOnFirstErrorWriteStatus.java @@ -0,0 +1,46 @@ +/* + * 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.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Map; + +/** + * This class can be used as WriteStatus when we want to fail fast and at the first available exception/error. + */ +public class FailOnFirstErrorWriteStatus extends WriteStatus { + private static final Logger LOG = LoggerFactory.getLogger(FailOnFirstErrorWriteStatus.class); + + public FailOnFirstErrorWriteStatus(Boolean trackSuccessRecords, Double failureFraction) { + super(trackSuccessRecords, failureFraction); + } + + @Override + public void markFailure(HoodieRecord record, Throwable t, Option> optionalRecordMetadata) { + LOG.error(String.format("Error writing record %s with data %s and optionalRecordMetadata %s", record, record.getData(), + optionalRecordMetadata.orElse(Collections.emptyMap()), t)); + throw new HoodieException("Error writing record " + record + ": " + t.getMessage()); + } +} 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 ddc5d41448399..56ea142a06068 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 @@ -436,6 +436,8 @@ private Stream getCommitInstantsToArchive() throws IOException { // 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()); @@ -451,7 +453,7 @@ private Stream getCommitInstantsToArchive() throws IOException { return !(firstSavepoint.isPresent() && compareTimestamps(firstSavepoint.get().getTimestamp(), LESSER_THAN_OR_EQUALS, s.getTimestamp())); } }).filter(s -> { - // Ensure commits >= the 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); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java index b306d6c540017..54e88fcca225b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java @@ -101,7 +101,18 @@ public void markSuccess(HoodieRecord record, Option> optiona String eventTimeVal = optionalRecordMetadata.get().getOrDefault(METADATA_EVENT_TIME_KEY, null); try { if (!StringUtils.isNullOrEmpty(eventTimeVal)) { - long eventTime = DateTimeUtils.parseDateTime(eventTimeVal).toEpochMilli(); + int length = eventTimeVal.length(); + long millisEventTime; + // eventTimeVal in seconds unit + if (length == 10) { + millisEventTime = Long.parseLong(eventTimeVal) * 1000; + } else if (length == 13) { + // eventTimeVal in millis unit + millisEventTime = Long.parseLong(eventTimeVal); + } else { + throw new IllegalArgumentException("not support event_time format:" + eventTimeVal); + } + long eventTime = DateTimeUtils.parseDateTime(Long.toString(millisEventTime)).toEpochMilli(); stat.setMinEventTime(eventTime); stat.setMaxEventTime(eventTime); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java index e07bde915a089..6fcc975004564 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java @@ -95,7 +95,9 @@ public void startServer() throws IOException { .earlyConflictDetectionCheckCommitConflict(writeConfig.earlyConflictDetectionCheckCommitConflict()) .asyncConflictDetectorInitialDelayMs(writeConfig.getAsyncConflictDetectorInitialDelayMs()) .asyncConflictDetectorPeriodMs(writeConfig.getAsyncConflictDetectorPeriodMs()) - .earlyConflictDetectionMaxAllowableHeartbeatIntervalInMs(writeConfig.getHoodieClientHeartbeatIntervalInMs()); + .earlyConflictDetectionMaxAllowableHeartbeatIntervalInMs( + writeConfig.getHoodieClientHeartbeatIntervalInMs() + * writeConfig.getHoodieClientHeartbeatTolerableMisses()); } server = new TimelineService(context, hadoopConf.newCopy(), timelineServiceConfBuilder.build(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java new file mode 100644 index 0000000000000..a15a4cc533c82 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/BucketIndexConcurrentFileWritesConflictResolutionStrategy.java @@ -0,0 +1,61 @@ +/* + * 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.transaction; + +import org.apache.hudi.index.bucket.BucketIdentifier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy} using bucket index. + */ +public class BucketIndexConcurrentFileWritesConflictResolutionStrategy + extends SimpleConcurrentFileWritesConflictResolutionStrategy { + private static final Logger LOG = LoggerFactory.getLogger(BucketIndexConcurrentFileWritesConflictResolutionStrategy.class); + + @Override + public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { + // TODO : UUID's can clash even for insert/insert, handle that case. + Set partitionBucketIdSetForFirstInstant = thisOperation + .getMutatedPartitionAndFileIds() + .stream() + .map(partitionAndFileId -> + BucketIdentifier.partitionBucketIdStr(partitionAndFileId.getLeft(), BucketIdentifier.bucketIdFromFileId(partitionAndFileId.getRight())) + ).collect(Collectors.toSet()); + Set partitionBucketIdSetForSecondInstant = otherOperation + .getMutatedPartitionAndFileIds() + .stream() + .map(partitionAndFileId -> + BucketIdentifier.partitionBucketIdStr(partitionAndFileId.getLeft(), BucketIdentifier.bucketIdFromFileId(partitionAndFileId.getRight())) + ).collect(Collectors.toSet()); + Set intersection = new HashSet<>(partitionBucketIdSetForFirstInstant); + intersection.retainAll(partitionBucketIdSetForSecondInstant); + if (!intersection.isEmpty()) { + LOG.info("Found conflicting writes between first operation = " + thisOperation + + ", second operation = " + otherOperation + " , intersecting bucket ids " + intersection); + return true; + } + return false; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java index 4e69b69cf4712..20874849bc30f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import java.io.IOException; import java.util.Collections; @@ -38,7 +39,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; -import static org.apache.hudi.common.util.CommitUtils.getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord; +import static org.apache.hudi.common.util.CommitUtils.getPartitionAndFileIdWithoutSuffixFromSpecificRecord; /** * This class is used to hold all information used to identify how to resolve conflicts between instants. @@ -53,7 +54,7 @@ public class ConcurrentOperation { private final String actionState; private final String actionType; private final String instantTime; - private Set mutatedFileIds = Collections.EMPTY_SET; + private Set> mutatedPartitionAndFileIds = Collections.emptySet(); public ConcurrentOperation(HoodieInstant instant, HoodieTableMetaClient metaClient) throws IOException { this.metadataWrapper = new HoodieMetadataWrapper(MetadataConversionUtils.createMetaWrapper(instant, metaClient)); @@ -89,8 +90,8 @@ public WriteOperationType getOperationType() { return operationType; } - public Set getMutatedFileIds() { - return mutatedFileIds; + public Set> getMutatedPartitionAndFileIds() { + return mutatedPartitionAndFileIds; } public Option getCommitMetadataOption() { @@ -102,21 +103,21 @@ private void init(HoodieInstant instant) { switch (getInstantActionType()) { case COMPACTION_ACTION: this.operationType = WriteOperationType.COMPACT; - this.mutatedFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieCompactionPlan().getOperations() + this.mutatedPartitionAndFileIds = this.metadataWrapper.getMetadataFromTimeline().getHoodieCompactionPlan().getOperations() .stream() - .map(op -> op.getFileId()) + .map(operation -> Pair.of(operation.getPartitionPath(), operation.getFileId())) .collect(Collectors.toSet()); break; case COMMIT_ACTION: case DELTA_COMMIT_ACTION: - this.mutatedFileIds = getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata() - .getPartitionToWriteStats()).keySet(); + this.mutatedPartitionAndFileIds = getPartitionAndFileIdWithoutSuffixFromSpecificRecord(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata() + .getPartitionToWriteStats()); this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieCommitMetadata().getOperationType()); break; case REPLACE_COMMIT_ACTION: if (instant.isCompleted()) { - this.mutatedFileIds = getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord( - this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats()).keySet(); + this.mutatedPartitionAndFileIds = getPartitionAndFileIdWithoutSuffixFromSpecificRecord( + this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getPartitionToWriteStats()); this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieReplaceCommitMetadata().getOperationType()); } else { // we need to have different handling for requested and inflight replacecommit because @@ -127,16 +128,16 @@ private void init(HoodieInstant instant) { if (instant.isRequested()) { // for insert_overwrite/insert_overwrite_table clusteringPlan will be empty if (requestedReplaceMetadata != null && requestedReplaceMetadata.getClusteringPlan() != null) { - this.mutatedFileIds = getFileIdsFromRequestedReplaceMetadata(requestedReplaceMetadata); + this.mutatedPartitionAndFileIds = getPartitionAndFileIdsFromRequestedReplaceMetadata(requestedReplaceMetadata); this.operationType = WriteOperationType.CLUSTER; } } else { if (inflightCommitMetadata != null) { - this.mutatedFileIds = getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(inflightCommitMetadata.getPartitionToWriteStats()).keySet(); + this.mutatedPartitionAndFileIds = getPartitionAndFileIdWithoutSuffixFromSpecificRecord(inflightCommitMetadata.getPartitionToWriteStats()); this.operationType = WriteOperationType.fromValue(this.metadataWrapper.getMetadataFromTimeline().getHoodieInflightReplaceMetadata().getOperationType()); } else if (requestedReplaceMetadata != null) { // inflight replacecommit metadata is empty due to clustering, read fileIds from requested replacecommit - this.mutatedFileIds = getFileIdsFromRequestedReplaceMetadata(requestedReplaceMetadata); + this.mutatedPartitionAndFileIds = getPartitionAndFileIdsFromRequestedReplaceMetadata(requestedReplaceMetadata); this.operationType = WriteOperationType.CLUSTER; } // NOTE: it cannot be the case that instant is inflight, and both the requested and inflight replacecommit metadata are empty @@ -151,7 +152,7 @@ private void init(HoodieInstant instant) { case COMMIT_ACTION: case DELTA_COMMIT_ACTION: case LOG_COMPACTION_ACTION: - this.mutatedFileIds = CommitUtils.getFileIdWithoutSuffixAndRelativePaths(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()).keySet(); + this.mutatedPartitionAndFileIds = CommitUtils.getPartitionAndFileIdWithoutSuffix(this.metadataWrapper.getCommitMetadata().getPartitionToWriteStats()); this.operationType = this.metadataWrapper.getCommitMetadata().getOperationType(); break; default: @@ -160,12 +161,12 @@ private void init(HoodieInstant instant) { } } - private static Set getFileIdsFromRequestedReplaceMetadata(HoodieRequestedReplaceMetadata requestedReplaceMetadata) { + private static Set> getPartitionAndFileIdsFromRequestedReplaceMetadata(HoodieRequestedReplaceMetadata requestedReplaceMetadata) { return requestedReplaceMetadata .getClusteringPlan().getInputGroups() .stream() .flatMap(ig -> ig.getSlices().stream()) - .map(file -> file.getFileId()) + .map(fileSlice -> Pair.of(fileSlice.getPartitionPath(), fileSlice.getFileId())) .collect(Collectors.toSet()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index 84fac2db004a8..7ed6d51038c08 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -42,13 +42,12 @@ public class DirectMarkerTransactionManager extends TransactionManager { private final String filePath; public DirectMarkerTransactionManager(HoodieWriteConfig config, FileSystem fs, String partitionPath, String fileId) { - super(new LockManager(config, fs, createUpdatedLockProps(config, partitionPath, fileId)), - config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()); + super(new LockManager(config, fs, createUpdatedLockProps(config, partitionPath, fileId)), config.isLockRequired()); this.filePath = partitionPath + "/" + fileId; } public void beginTransaction(String newTxnOwnerInstantTime) { - if (isOptimisticConcurrencyControlEnabled) { + if (isLockRequired) { LOG.info("Transaction starting for " + newTxnOwnerInstantTime + " and " + filePath); lockManager.lock(); @@ -58,7 +57,7 @@ public void beginTransaction(String newTxnOwnerInstantTime) { } public void endTransaction(String currentTxnOwnerInstantTime) { - if (isOptimisticConcurrencyControlEnabled) { + if (isLockRequired) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstantTime + " for " + filePath); if (reset(Option.of(getInstant(currentTxnOwnerInstantTime)), Option.empty(), Option.empty())) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java index a622486b17d91..22bcd8d0e0cc9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieWriteConflictException; import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; @@ -72,10 +73,10 @@ public Stream getCandidateInstants(HoodieActiveTimeline activeTim @Override public boolean hasConflict(ConcurrentOperation thisOperation, ConcurrentOperation otherOperation) { // TODO : UUID's can clash even for insert/insert, handle that case. - Set fileIdsSetForFirstInstant = thisOperation.getMutatedFileIds(); - Set fileIdsSetForSecondInstant = otherOperation.getMutatedFileIds(); - Set intersection = new HashSet<>(fileIdsSetForFirstInstant); - intersection.retainAll(fileIdsSetForSecondInstant); + Set> partitionAndFileIdsSetForFirstInstant = thisOperation.getMutatedPartitionAndFileIds(); + Set> partitionAndFileIdsSetForSecondInstant = otherOperation.getMutatedPartitionAndFileIds(); + Set> intersection = new HashSet<>(partitionAndFileIdsSetForFirstInstant); + intersection.retainAll(partitionAndFileIdsSetForSecondInstant); if (!intersection.isEmpty()) { LOG.info("Found conflicting writes between first operation = " + thisOperation + ", second operation = " + otherOperation + " , intersecting file ids " + intersection); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java index 7fddf8a944b06..d1163abac598a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/TransactionManager.java @@ -37,23 +37,22 @@ public class TransactionManager implements Serializable { protected static final Logger LOG = LogManager.getLogger(TransactionManager.class); protected final LockManager lockManager; - protected final boolean isOptimisticConcurrencyControlEnabled; + protected final boolean isLockRequired; protected Option currentTxnOwnerInstant = Option.empty(); private Option lastCompletedTxnOwnerInstant = Option.empty(); public TransactionManager(HoodieWriteConfig config, FileSystem fs) { - this(new LockManager(config, fs), - config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()); + this(new LockManager(config, fs), config.isLockRequired()); } - protected TransactionManager(LockManager lockManager, boolean isOptimisticConcurrencyControlEnabled) { + protected TransactionManager(LockManager lockManager, boolean isLockRequired) { this.lockManager = lockManager; - this.isOptimisticConcurrencyControlEnabled = isOptimisticConcurrencyControlEnabled; + this.isLockRequired = isLockRequired; } public void beginTransaction(Option newTxnOwnerInstant, Option lastCompletedTxnOwnerInstant) { - if (isOptimisticConcurrencyControlEnabled) { + if (isLockRequired) { LOG.info("Transaction starting for " + newTxnOwnerInstant + " with latest completed transaction instant " + lastCompletedTxnOwnerInstant); lockManager.lock(); @@ -64,7 +63,7 @@ public void beginTransaction(Option newTxnOwnerInstant, } public void endTransaction(Option currentTxnOwnerInstant) { - if (isOptimisticConcurrencyControlEnabled) { + if (isLockRequired) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstant); if (reset(currentTxnOwnerInstant, Option.empty(), Option.empty())) { lockManager.unlock(); @@ -85,7 +84,7 @@ protected synchronized boolean reset(Option callerInstant, } public void close() { - if (isOptimisticConcurrencyControlEnabled) { + if (isLockRequired) { lockManager.close(); LOG.info("Transaction manager closed"); } @@ -103,7 +102,7 @@ public Option getCurrentTransactionOwner() { return currentTxnOwnerInstant; } - public boolean isOptimisticConcurrencyControlEnabled() { - return isOptimisticConcurrencyControlEnabled; + public boolean isLockRequired() { + return isLockRequired; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java index efa644f4b0dce..8956f72560fef 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java @@ -24,8 +24,10 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.lock.LockState; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -41,6 +43,8 @@ import java.io.IOException; import java.io.Serializable; import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY; @@ -69,13 +73,17 @@ public FileSystemBasedLockProvider(final LockConfiguration lockConfiguration, fi String lockDirectory = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null); if (StringUtils.isNullOrEmpty(lockDirectory)) { lockDirectory = lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key()) - + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; + + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; } this.lockTimeoutMinutes = lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY); this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME); this.lockInfo = new LockInfo(); this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); this.fs = FSUtils.getFs(this.lockFile.toString(), configuration); + List customSupportedFSs = lockConfiguration.getConfig().getStringList(HoodieCommonConfig.HOODIE_FS_ATOMIC_CREATION_SUPPORT.key(), ",", new ArrayList<>()); + if (!customSupportedFSs.contains(this.fs.getScheme()) && !StorageSchemes.isAtomicCreationSupported(this.fs.getScheme())) { + throw new HoodieLockException("Unsupported scheme :" + this.fs.getScheme() + ", since this fs can not support atomic creation"); + } } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java index 2a60473c82c8d..904c99019ec49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/InProcessLockProvider.java @@ -124,7 +124,6 @@ public void close() { lock.writeLock().unlock(); } LOG.info(getLogMessage(LockState.ALREADY_RELEASED)); - LOCK_INSTANCE_PER_BASEPATH.remove(basePath); } private String getLogMessage(LockState state) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java index effcdf091ae2f..3918f2ed74bd2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java @@ -68,39 +68,37 @@ public LockManager(HoodieWriteConfig writeConfig, FileSystem fs, TypedProperties } public void lock() { - if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { - LockProvider lockProvider = getLockProvider(); - int retryCount = 0; - boolean acquired = false; - while (retryCount <= maxRetries) { + LockProvider lockProvider = getLockProvider(); + int retryCount = 0; + boolean acquired = false; + while (retryCount <= maxRetries) { + try { + metrics.startLockApiTimerContext(); + acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS); + if (acquired) { + metrics.updateLockAcquiredMetric(); + break; + } + metrics.updateLockNotAcquiredMetric(); + LOG.info("Retrying to acquire lock. Current lock owner information : " + lockProvider.getCurrentOwnerLockInfo()); + Thread.sleep(maxWaitTimeInMs); + } catch (HoodieLockException | InterruptedException e) { + metrics.updateLockNotAcquiredMetric(); + if (retryCount >= maxRetries) { + throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock(), e); + } try { - metrics.startLockApiTimerContext(); - acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS); - if (acquired) { - metrics.updateLockAcquiredMetric(); - break; - } - metrics.updateLockNotAcquiredMetric(); - LOG.info("Retrying to acquire lock. Current lock owner information : " + lockProvider.getCurrentOwnerLockInfo()); Thread.sleep(maxWaitTimeInMs); - } catch (HoodieLockException | InterruptedException e) { - metrics.updateLockNotAcquiredMetric(); - if (retryCount >= maxRetries) { - throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock(), e); - } - try { - Thread.sleep(maxWaitTimeInMs); - } catch (InterruptedException ex) { - // ignore InterruptedException here - } - } finally { - retryCount++; + } catch (InterruptedException ex) { + // ignore InterruptedException here } - } - if (!acquired) { - throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock()); + } finally { + retryCount++; } } + if (!acquired) { + throw new HoodieLockException("Unable to acquire lock, lock object " + lockProvider.getLock()); + } } /** @@ -108,10 +106,9 @@ public void lock() { * and tries to call unlock() */ public void unlock() { - if (writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { - getLockProvider().unlock(); - metrics.updateLockHeldTimerMetrics(); - } + getLockProvider().unlock(); + metrics.updateLockHeldTimerMetrics(); + close(); } public synchronized LockProvider getLockProvider() { 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..92c2065457e0a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.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; +import java.util.stream.Stream; + +/** + * 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 + Stream.concat(fileSystemView.getPendingCompactionOperations(), fileSystemView.getPendingLogCompactionOperations()) + .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/HoodieArchivalConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java index 681ca20baeebf..fdfdab5897afe 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java @@ -63,7 +63,14 @@ public class HoodieArchivalConfig extends HoodieConfig { public static final ConfigProperty DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.archive.delete.parallelism") .defaultValue(100) - .withDocumentation("Parallelism for deleting archived hoodie commits."); + .withDocumentation("When performing archival operation, Hudi needs to delete the files of " + + "the archived instants in the active timeline in .hoodie folder. The file deletion " + + "also happens after merging small archived files into larger ones if enabled. " + + "This config limits the Spark parallelism for deleting files in both cases, i.e., " + + "parallelism of deleting files does not go above the configured value and the " + + "parallelism is the number of files to delete if smaller than the " + + "configured value. If you see that the file deletion in archival operation is slow " + + "because of the limited parallelism, you can increase this to tune the performance."); public static final ConfigProperty MIN_COMMITS_TO_KEEP = ConfigProperty .key("hoodie.keep.min.commits") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java index 0b9116b01c9cc..a6ee8ab41606c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java @@ -96,7 +96,15 @@ public class HoodieBootstrapConfig extends HoodieConfig { .key("hoodie.bootstrap.parallelism") .defaultValue("1500") .sinceVersion("0.6.0") - .withDocumentation("Parallelism value to be used to bootstrap data into hudi"); + .withDocumentation("For metadata-only bootstrap, Hudi parallelizes the operation so that " + + "each table partition is handled by one Spark task. This config limits the number " + + "of parallelism. We pick the configured parallelism if the number of table partitions " + + "is larger than this configured value. The parallelism is assigned to the number of " + + "table partitions if it is smaller than the configured value. For full-record " + + "bootstrap, i.e., BULK_INSERT operation of the records, this configured value is " + + "passed as the BULK_INSERT shuffle parallelism (`hoodie.bulkinsert.shuffle.parallelism`), " + + "determining the BULK_INSERT write behavior. If you see that the bootstrap is slow " + + "due to the limited parallelism, you can increase this."); public static final ConfigProperty PARTITION_SELECTOR_REGEX_PATTERN = ConfigProperty .key("hoodie.bootstrap.mode.selector.regex") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java index 30289e1acbab4..c1b66a371d19c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCleanConfig.java @@ -29,6 +29,7 @@ import org.apache.hudi.table.action.clean.CleaningTriggerStrategy; import javax.annotation.concurrent.Immutable; + import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -120,7 +121,17 @@ public class HoodieCleanConfig extends HoodieConfig { public static final ConfigProperty CLEANER_PARALLELISM_VALUE = ConfigProperty .key("hoodie.cleaner.parallelism") .defaultValue("200") - .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow."); + .withDocumentation("This config controls the behavior of both the cleaning plan and " + + "cleaning execution. Deriving the cleaning plan is parallelized at the table " + + "partition level, i.e., each table partition is processed by one Spark task to figure " + + "out the files to clean. The cleaner picks the configured parallelism if the number " + + "of table partitions is larger than this configured value. The parallelism is " + + "assigned to the number of table partitions if it is smaller than the configured value. " + + "The clean execution, i.e., the file deletion, is parallelized at file level, which " + + "is the unit of Spark task distribution. Similarly, the actual parallelism cannot " + + "exceed the configured value if the number of files is larger. If cleaning plan or " + + "execution is slow due to limited parallelism, you can increase this to tune the " + + "performance.."); public static final ConfigProperty ALLOW_MULTIPLE_CLEANS = ConfigProperty .key("hoodie.clean.allow.multiple") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java index e22bf1e43d136..e37ff3c46bf05 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy; @@ -188,11 +189,11 @@ public class HoodieCompactionConfig extends HoodieConfig { .withDocumentation("Log compaction can be scheduled if the no. of log blocks crosses this threshold value. " + "This is effective only when log compaction is enabled via " + INLINE_LOG_COMPACT.key()); - public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = ConfigProperty - .key("hoodie.log.record.reader.use.scanV2") + public static final ConfigProperty ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN = ConfigProperty + .key("hoodie" + HoodieMetadataConfig.OPTIMIZED_LOG_BLOCKS_SCAN) .defaultValue("false") .sinceVersion("0.13.0") - .withDocumentation("ScanV2 logic address all the multiwriter challenges while appending to log files. " + .withDocumentation("New optimized scan for log blocks that handles all multi-writer use-cases while appending to log files. " + "It also differentiates original blocks written by ingestion writers and compacted blocks written log compaction."); /** @deprecated Use {@link #INLINE_COMPACT} and its methods instead */ @@ -432,8 +433,8 @@ public Builder withLogCompactionBlocksThreshold(String logCompactionBlocksThresh return this; } - public Builder withLogRecordReaderScanV2(String useLogRecordReaderScanV2) { - compactionConfig.setValue(USE_LOG_RECORD_READER_SCAN_V2, useLogRecordReaderScanV2); + public Builder withEnableOptimizedLogBlocksScan(String enableOptimizedLogBlocksScan) { + compactionConfig.setValue(ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN, enableOptimizedLogBlocksScan); return this; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java index d52c407028a21..359b9ab6a5289 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java @@ -31,6 +31,7 @@ @ConfigClassProperty(name = "HBase Index Configs", groupName = ConfigGroups.Names.WRITE_CLIENT, + subGroupName = ConfigGroups.SubGroupNames.INDEX, description = "Configurations that control indexing behavior " + "(when HBase based indexing is enabled), which tags incoming " + "records as either inserts or updates to older records.") diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java index 87e9ae576dbee..3152ec34942c2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java @@ -59,10 +59,11 @@ * Indexing related config. */ @Immutable -@ConfigClassProperty(name = "Index Configs", +@ConfigClassProperty(name = "Common Index Configs", groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Configurations that control indexing behavior, " - + "which tags incoming records as either inserts or updates to older records.") + subGroupName = ConfigGroups.SubGroupNames.INDEX, + areCommonConfigs = true, + description = "") public class HoodieIndexConfig extends HoodieConfig { private static final Logger LOG = LogManager.getLogger(HoodieIndexConfig.class); @@ -113,7 +114,10 @@ public class HoodieIndexConfig extends HoodieConfig { .defaultValue("0") .withDocumentation("Only applies if index type is BLOOM. " + "This is the amount of parallelism for index lookup, which involves a shuffle. " - + "By default, this is auto computed based on input workload characteristics."); + + "By default, this is auto computed based on input workload characteristics. " + + "If the parallelism is explicitly configured by the user, the user-configured " + + "value is used in defining the actual parallelism. If the indexing stage is slow " + + "due to the limited parallelism, you can increase this to tune the performance."); public static final ConfigProperty BLOOM_INDEX_PRUNE_BY_RANGES = ConfigProperty .key("hoodie.bloom.index.prune.by.ranges") @@ -179,13 +183,21 @@ public class HoodieIndexConfig extends HoodieConfig { .key("hoodie.simple.index.parallelism") .defaultValue("100") .withDocumentation("Only applies if index type is SIMPLE. " - + "This is the amount of parallelism for index lookup, which involves a Spark Shuffle"); + + "This limits the parallelism of fetching records from the base files of affected " + + "partitions. The index picks the configured parallelism if the number of base " + + "files is larger than this configured value; otherwise, the number of base files " + + "is used as the parallelism. If the indexing stage is slow due to the limited " + + "parallelism, you can increase this to tune the performance."); public static final ConfigProperty GLOBAL_SIMPLE_INDEX_PARALLELISM = ConfigProperty .key("hoodie.global.simple.index.parallelism") .defaultValue("100") .withDocumentation("Only applies if index type is GLOBAL_SIMPLE. " - + "This is the amount of parallelism for index lookup, which involves a Spark Shuffle"); + + "This limits the parallelism of fetching records from the base files of all table " + + "partitions. The index picks the configured parallelism if the number of base " + + "files is larger than this configured value; otherwise, the number of base files " + + "is used as the parallelism. If the indexing stage is slow due to the limited " + + "parallelism, you can increase this to tune the performance."); // 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter. // 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java index 2c66ac8e68f91..62aa66942756e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java @@ -17,6 +17,7 @@ package org.apache.hudi.config; +import org.apache.hudi.client.transaction.BucketIndexConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.ConflictResolutionStrategy; import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy; import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; @@ -26,6 +27,7 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.lock.LockProvider; import org.apache.hudi.common.util.Option; +import org.apache.hudi.index.HoodieIndex; import java.io.File; import java.io.FileReader; @@ -58,11 +60,11 @@ /** * Hoodie Configs for Locks. */ -@ConfigClassProperty(name = "Locks Configurations", +@ConfigClassProperty(name = "Common Lock Configurations", groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Configs that control locking mechanisms required for concurrency control " - + " between writers to a Hudi table. Concurrency between Hudi's own table services " - + " are auto managed internally.") + subGroupName = ConfigGroups.SubGroupNames.LOCK, + areCommonConfigs = true, + description = "") public class HoodieLockConfig extends HoodieConfig { public static final ConfigProperty LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = ConfigProperty @@ -81,7 +83,7 @@ public class HoodieLockConfig extends HoodieConfig { public static final ConfigProperty LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = ConfigProperty .key(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY) - .defaultValue(String.valueOf(2000L)) + .defaultValue(String.valueOf(5000L)) .sinceVersion("0.8.0") .withDocumentation("Amount of time to wait between retries on the lock provider by the lock manager"); @@ -185,6 +187,13 @@ public class HoodieLockConfig extends HoodieConfig { public static final ConfigProperty WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME = ConfigProperty .key(LOCK_PREFIX + "conflict.resolution.strategy") .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()) + .withInferFunction(hoodieConfig -> { + if (HoodieIndex.IndexType.BUCKET.name().equalsIgnoreCase(hoodieConfig.getStringOrDefault(HoodieIndexConfig.INDEX_TYPE, null))) { + return Option.of(BucketIndexConcurrentFileWritesConflictResolutionStrategy.class.getName()); + } else { + return Option.of(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName()); + } + }) .sinceVersion("0.8.0") .withDocumentation("Lock provider class name, this should be subclass of " + "org.apache.hudi.client.transaction.ConflictResolutionStrategy"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java index 09c2b09be0321..8be57c05d1c4e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java @@ -32,8 +32,9 @@ */ @ConfigClassProperty(name = "Write commit callback configs", groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Controls callback behavior into HTTP endpoints, to push " - + " notifications on commits on hudi tables.") + subGroupName = ConfigGroups.SubGroupNames.COMMIT_CALLBACK, + areCommonConfigs = true, + description = "") public class HoodieWriteCommitCallbackConfig extends HoodieConfig { public static final String CALLBACK_PREFIX = "hoodie.write.commit.callback."; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 2f36aa725e338..f61ba16a33c6c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -53,7 +53,8 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.queue.DisruptorWaitStrategyType; import org.apache.hudi.common.util.queue.ExecutorType; import org.apache.hudi.config.metrics.HoodieMetricsCloudWatchConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig; @@ -67,6 +68,7 @@ import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; +import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metrics.MetricsReporterType; import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite; import org.apache.hudi.table.RandomFileIdPrefixProvider; @@ -97,8 +99,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static org.apache.hudi.common.util.queue.ExecutorType.BOUNDED_IN_MEMORY; -import static org.apache.hudi.common.util.queue.ExecutorType.DISRUPTOR; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; +import static org.apache.hudi.common.util.queue.ExecutorType.SIMPLE; import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_POLICY; import static org.apache.hudi.table.marker.ConflictDetectionUtils.getDefaultEarlyConflictDetectionStrategy; @@ -158,23 +160,25 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator` " + "extract a key out of incoming records."); - public static final ConfigProperty EXECUTOR_TYPE = ConfigProperty + public static final ConfigProperty WRITE_EXECUTOR_TYPE = ConfigProperty .key("hoodie.write.executor.type") - .defaultValue(BOUNDED_IN_MEMORY.name()) - .withValidValues(BOUNDED_IN_MEMORY.name(), DISRUPTOR.name()) + .defaultValue(SIMPLE.name()) + .withValidValues(Arrays.stream(ExecutorType.values()).map(Enum::name).toArray(String[]::new)) .sinceVersion("0.13.0") .withDocumentation("Set executor which orchestrates concurrent producers and consumers communicating through a message queue." - + "BOUNDED_IN_MEMORY(default): Use LinkedBlockingQueue as a bounded in-memory queue, this queue will use extra lock to balance producers and consumer" + + "BOUNDED_IN_MEMORY: Use LinkedBlockingQueue as a bounded in-memory queue, this queue will use extra lock to balance producers and consumer" + "DISRUPTOR: Use disruptor which a lock free message queue as inner message, this queue may gain better writing performance if lock was the bottleneck. " - + "SIMPLE: Executor with no inner message queue and no inner lock. Consuming and writing records from iterator directly. Compared with BIM and DISRUPTOR, " + + "SIMPLE(default): Executor with no inner message queue and no inner lock. Consuming and writing records from iterator directly. Compared with BIM and DISRUPTOR, " + "this queue has no need for additional memory and cpu resources due to lock or multithreading, but also lost some benefits such as speed limit. " - + "Although DISRUPTOR_EXECUTOR and SIMPLE are still in experimental."); + + "Although DISRUPTOR is still experimental."); public static final ConfigProperty KEYGENERATOR_TYPE = ConfigProperty .key("hoodie.datasource.write.keygenerator.type") .defaultValue(KeyGeneratorType.SIMPLE.name()) .withDocumentation("Easily configure one the built-in key generators, instead of specifying the key generator class." - + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE"); + + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE. " + + "**Note** This is being actively worked on. Please use " + + "`hoodie.datasource.write.keygenerator.class` instead."); public static final ConfigProperty ROLLBACK_USING_MARKERS_ENABLE = ConfigProperty .key("hoodie.rollback.using.markers") @@ -223,19 +227,44 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty AVRO_SCHEMA_VALIDATE_ENABLE = ConfigProperty .key("hoodie.avro.schema.validate") - .defaultValue("true") + .defaultValue("false") .withDocumentation("Validate the schema used for the write against the latest schema, for backwards compatibility."); + public static final ConfigProperty SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP = ConfigProperty + .key("hoodie.datasource.write.schema.allow.auto.evolution.column.drop") + .defaultValue("false") + .sinceVersion("0.13.0") + .withDocumentation("Controls whether table's schema is allowed to automatically evolve when " + + "incoming batch's schema can have any of the columns dropped. By default, Hudi will not " + + "allow this kind of (auto) schema evolution. Set this config to true to allow table's " + + "schema to be updated automatically when columns are dropped from the new incoming batch."); + public static final ConfigProperty INSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.insert.shuffle.parallelism") .defaultValue("0") - .withDocumentation("Parallelism for inserting records into the table. Inserts can shuffle data before writing to tune file sizes and optimize the storage layout."); + .withDocumentation("Parallelism for inserting records into the table. Inserts can shuffle " + + "data before writing to tune file sizes and optimize the storage layout. Before " + + "0.13.0 release, if users do not configure it, Hudi would use 200 as the default " + + "shuffle parallelism. From 0.13.0 onwards Hudi by default automatically uses the " + + "parallelism deduced by Spark based on the source data. If the shuffle parallelism " + + "is explicitly configured by the user, the user-configured parallelism is " + + "used in defining the actual parallelism. If you observe small files from the insert " + + "operation, we suggest configuring this shuffle parallelism explicitly, so that the " + + "parallelism is around total_input_data_size/120MB."); public static final ConfigProperty BULKINSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.bulkinsert.shuffle.parallelism") .defaultValue("0") - .withDocumentation("For large initial imports using bulk_insert operation, controls the parallelism to use for sort modes or custom partitioning done" - + "before writing records to the table."); + .withDocumentation("For large initial imports using bulk_insert operation, controls the " + + "parallelism to use for sort modes or custom partitioning done before writing records " + + "to the table. Before 0.13.0 release, if users do not configure it, Hudi would use " + + "200 as the default shuffle parallelism. From 0.13.0 onwards Hudi by default " + + "automatically uses the parallelism deduced by Spark based on the source data or " + + "the parallelism based on the logical plan for row writer. If the shuffle parallelism " + + "is explicitly configured by the user, the user-configured parallelism is " + + "used in defining the actual parallelism. If you observe small files from the bulk insert " + + "operation, we suggest configuring this shuffle parallelism explicitly, so that the " + + "parallelism is around total_input_data_size/120MB."); public static final ConfigProperty BULKINSERT_USER_DEFINED_PARTITIONER_SORT_COLUMNS = ConfigProperty .key("hoodie.bulkinsert.user.defined.partitioner.sort.columns") @@ -253,33 +282,51 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty UPSERT_PARALLELISM_VALUE = ConfigProperty .key("hoodie.upsert.shuffle.parallelism") .defaultValue("0") - .withDocumentation("Parallelism to use for upsert operation on the table. Upserts can shuffle data to perform index lookups, file sizing, bin packing records optimally" - + "into file groups."); + .withDocumentation("Parallelism to use for upsert operation on the table. Upserts can " + + "shuffle data to perform index lookups, file sizing, bin packing records optimally " + + "into file groups. Before 0.13.0 release, " + + "if users do not configure it, Hudi would use 200 as the default " + + "shuffle parallelism. From 0.13.0 onwards Hudi by default automatically uses the " + + "parallelism deduced by Spark based on the source data. If the shuffle parallelism " + + "is explicitly configured by the user, the user-configured parallelism is " + + "used in defining the actual parallelism. If you observe small files from the upsert " + + "operation, we suggest configuring this shuffle parallelism explicitly, so that the " + + "parallelism is around total_input_data_size/120MB."); public static final ConfigProperty DELETE_PARALLELISM_VALUE = ConfigProperty .key("hoodie.delete.shuffle.parallelism") .defaultValue("0") - .withDocumentation("Parallelism used for “delete” operation. Delete operations also performs shuffles, similar to upsert operation."); + .withDocumentation("Parallelism used for delete operation. Delete operations also performs " + + "shuffles, similar to upsert operation. Before 0.13.0 release, " + + "if users do not configure it, Hudi would use 200 as the default " + + "shuffle parallelism. From 0.13.0 onwards Hudi by default automatically uses the " + + "parallelism deduced by Spark based on the source data. If the shuffle parallelism " + + "is explicitly configured by the user, the user-configured parallelism is " + + "used in defining the actual parallelism."); public static final ConfigProperty ROLLBACK_PARALLELISM_VALUE = ConfigProperty .key("hoodie.rollback.parallelism") .defaultValue("100") - .withDocumentation("Parallelism for rollback of commits. Rollbacks perform delete of files or logging delete blocks to file groups on storage in parallel."); + .withDocumentation("This config controls the parallelism for rollback of commits. " + + "Rollbacks perform deletion of files or logging delete blocks to file groups on " + + "storage in parallel. The configure value limits the parallelism so that the number " + + "of Spark tasks do not exceed the value. If rollback is slow due to the limited " + + "parallelism, you can increase this to tune the performance."); public static final ConfigProperty WRITE_BUFFER_LIMIT_BYTES_VALUE = ConfigProperty .key("hoodie.write.buffer.limit.bytes") .defaultValue(String.valueOf(4 * 1024 * 1024)) .withDocumentation("Size of in-memory buffer used for parallelizing network reads and lake storage writes."); - public static final ConfigProperty WRITE_DISRUPTOR_BUFFER_SIZE = ConfigProperty - .key("hoodie.write.executor.disruptor.buffer.size") + public static final ConfigProperty WRITE_EXECUTOR_DISRUPTOR_BUFFER_LIMIT_BYTES = ConfigProperty + .key("hoodie.write.executor.disruptor.buffer.limit.bytes") .defaultValue(String.valueOf(1024)) .sinceVersion("0.13.0") .withDocumentation("The size of the Disruptor Executor ring buffer, must be power of 2"); - public static final ConfigProperty WRITE_WAIT_STRATEGY = ConfigProperty + public static final ConfigProperty WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY = ConfigProperty .key("hoodie.write.executor.disruptor.wait.strategy") - .defaultValue("BLOCKING_WAIT") + .defaultValue(DisruptorWaitStrategyType.BLOCKING_WAIT.name()) .sinceVersion("0.13.0") .withDocumentation("Strategy employed for making Disruptor Executor wait on a cursor. Other options are " + "SLEEPING_WAIT, it attempts to be conservative with CPU usage by using a simple busy wait loop" @@ -581,7 +628,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty ASYNC_CONFLICT_DETECTOR_INITIAL_DELAY_MS = ConfigProperty .key(CONCURRENCY_PREFIX + "async.conflict.detector.initial_delay_ms") - .defaultValue(30000L) + .defaultValue(0L) .sinceVersion("0.13.0") .withDocumentation("Used for timeline-server-based markers with " + "`AsyncTimelineServerBasedDetectionStrategy`. " @@ -602,6 +649,12 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Whether to enable commit conflict checking or not during early " + "conflict detection."); + public static final ConfigProperty SENSITIVE_CONFIG_KEYS_FILTER = ConfigProperty + .key("hoodie.sensitive.config.keys") + .defaultValue("ssl,tls,sasl,auth,credentials") + .withDocumentation("Comma separated list of filters for sensitive config keys. Delta Streamer " + + "will not print any configuration which contains the configured filter. For example with " + + "a configured filter `ssl`, value for config `ssl.trustore.location` would be masked."); private ConsistencyGuardConfig consistencyGuardConfig; private FileSystemRetryConfig fileSystemRetryConfig; @@ -1024,7 +1077,7 @@ public String getBasePath() { } public HoodieRecordMerger getRecordMerger() { - List mergers = getSplitStringsOrDefault(RECORD_MERGER_IMPLS).stream() + List mergers = StringUtils.split(getStringOrDefault(RECORD_MERGER_IMPLS), ",").stream() .map(String::trim) .distinct() .collect(Collectors.toList()); @@ -1085,6 +1138,10 @@ public boolean shouldValidateAvroSchema() { return getBoolean(AVRO_SCHEMA_VALIDATE_ENABLE); } + public boolean shouldAllowAutoEvolutionColumnDrop() { + return getBooleanOrDefault(SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP); + } + public String getTableName() { return getString(TBL_NAME); } @@ -1107,7 +1164,7 @@ public String getKeyGeneratorClass() { } public ExecutorType getExecutorType() { - return ExecutorType.valueOf(getStringOrDefault(EXECUTOR_TYPE).toUpperCase(Locale.ROOT)); + return ExecutorType.valueOf(getStringOrDefault(WRITE_EXECUTOR_TYPE).toUpperCase(Locale.ROOT)); } public boolean isCDCEnabled() { @@ -1175,12 +1232,12 @@ public int getWriteBufferLimitBytes() { return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE)); } - public Option getWriteExecutorWaitStrategy() { - return Option.of(getString(WRITE_WAIT_STRATEGY)); + public String getWriteExecutorDisruptorWaitStrategy() { + return getStringOrDefault(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY); } - public Option getDisruptorWriteBufferSize() { - return Option.of(Integer.parseInt(getStringOrDefault(WRITE_DISRUPTOR_BUFFER_SIZE))); + public Integer getWriteExecutorDisruptorWriteBufferLimitBytes() { + return Integer.parseInt(getStringOrDefault(WRITE_EXECUTOR_DISRUPTOR_BUFFER_LIMIT_BYTES)); } public boolean shouldCombineBeforeInsert() { @@ -1305,8 +1362,8 @@ public int getLogCompactionBlocksThreshold() { return getInt(HoodieCompactionConfig.LOG_COMPACTION_BLOCKS_THRESHOLD); } - public boolean useScanV2ForLogRecordReader() { - return getBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2); + public boolean enableOptimizedLogBlocksScan() { + return getBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN); } public HoodieCleaningPolicy getCleanerPolicy() { @@ -1862,8 +1919,8 @@ public int getParquetPageSize() { return getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE); } - public int getLogFileDataBlockMaxSize() { - return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_MAX_SIZE); + public long getLogFileDataBlockMaxSize() { + return getLong(HoodieStorageConfig.LOGFILE_DATA_BLOCK_MAX_SIZE); } public double getParquetCompressionRatio() { @@ -1987,7 +2044,7 @@ public ApiSite getDatadogApiSite() { public String getDatadogApiKey() { if (props.containsKey(HoodieMetricsDatadogConfig.API_KEY.key())) { return getString(HoodieMetricsDatadogConfig.API_KEY); - + } else { Supplier apiKeySupplier = ReflectionUtils.loadClass( getString(HoodieMetricsDatadogConfig.API_KEY_SUPPLIER)); @@ -2194,7 +2251,7 @@ public Integer getHoodieClientHeartbeatTolerableMisses() { * File listing metadata configs. */ public boolean isMetadataTableEnabled() { - return metadataConfig.enabled(); + return getBooleanOrDefault(HoodieMetadataConfig.ENABLE); } public int getMetadataInsertParallelism() { @@ -2345,6 +2402,20 @@ public boolean areReleaseResourceEnabled() { return getBooleanOrDefault(RELEASE_RESOURCE_ENABLE); } + /** + * Returns whether the explicit guard of lock is required. + */ + public boolean isLockRequired() { + return !isDefaultLockProvider() || getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + } + + /** + * Returns whether the lock provider is default. + */ + private boolean isDefaultLockProvider() { + return HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue().equals(getLockProviderClass()); + } + /** * Layout configs. */ @@ -2363,7 +2434,7 @@ public boolean isMetaserverEnabled() { * CDC supplemental logging mode. */ public HoodieCDCSupplementalLoggingMode getCDCSupplementalLoggingMode() { - return HoodieCDCSupplementalLoggingMode.parse( + return HoodieCDCSupplementalLoggingMode.valueOf( getStringOrDefault(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE)); } @@ -2450,6 +2521,11 @@ public Builder withAvroSchemaValidate(boolean enable) { return this; } + public Builder withAllowAutoEvolutionColumnDrop(boolean shouldAllowDroppedColumns) { + writeConfig.setValue(SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP, String.valueOf(shouldAllowDroppedColumns)); + return this; + } + public Builder forTable(String tableName) { writeConfig.setValue(TBL_NAME, tableName); return this; @@ -2481,7 +2557,7 @@ public Builder withKeyGenerator(String keyGeneratorClass) { } public Builder withExecutorType(String executorClass) { - writeConfig.setValue(EXECUTOR_TYPE, executorClass); + writeConfig.setValue(WRITE_EXECUTOR_TYPE, executorClass); return this; } @@ -2536,13 +2612,13 @@ public Builder withWriteBufferLimitBytes(int writeBufferLimit) { return this; } - public Builder withWriteWaitStrategy(String waitStrategy) { - writeConfig.setValue(WRITE_WAIT_STRATEGY, String.valueOf(waitStrategy)); + public Builder withWriteExecutorDisruptorWaitStrategy(String waitStrategy) { + writeConfig.setValue(WRITE_EXECUTOR_DISRUPTOR_WAIT_STRATEGY, String.valueOf(waitStrategy)); return this; } - public Builder withWriteBufferSize(int size) { - writeConfig.setValue(WRITE_DISRUPTOR_BUFFER_SIZE, String.valueOf(size)); + public Builder withWriteExecutorDisruptorWriteBufferLimitBytes(long size) { + writeConfig.setValue(WRITE_EXECUTOR_DISRUPTOR_BUFFER_LIMIT_BYTES, String.valueOf(size)); return this; } @@ -2881,41 +2957,34 @@ protected void setDefaults() { // isLockProviderPropertySet must be fetched before setting defaults of HoodieLockConfig final TypedProperties writeConfigProperties = writeConfig.getProps(); - final boolean isLockProviderPropertySet = writeConfigProperties.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME) - || writeConfigProperties.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP); + final boolean isLockProviderPropertySet = writeConfigProperties.containsKey(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key()); writeConfig.setDefaultOnCondition(!isLockConfigSet, HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build()); autoAdjustConfigsForConcurrencyMode(isLockProviderPropertySet); } + private boolean isLockRequiredForSingleWriter() { + // When metadata table is enabled, lock provider must be used for + // single writer with async table services. + // Async table services can update the metadata table and a lock provider is + // needed to guard against any concurrent table write operations. If user has + // not configured any lock provider, let's use the InProcess lock provider. + return writeConfig.isMetadataTableEnabled() && writeConfig.areAnyTableServicesAsync() + && !writeConfig.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl(); + } + private void autoAdjustConfigsForConcurrencyMode(boolean isLockProviderPropertySet) { - if (writeConfig.isAutoAdjustLockConfigs()) { + if (!isLockProviderPropertySet && writeConfig.isAutoAdjustLockConfigs() && isLockRequiredForSingleWriter()) { // auto adjustment is required only for deltastreamer and spark streaming where async table services can be executed in the same JVM. - boolean isMetadataTableEnabled = writeConfig.getBoolean(HoodieMetadataConfig.ENABLE); - - if (isMetadataTableEnabled) { - // When metadata table is enabled, optimistic concurrency control must be used for - // single writer with async table services. - // Async table services can update the metadata table and a lock provider is - // needed to guard against any concurrent table write operations. If user has - // not configured any lock provider, let's use the InProcess lock provider. - boolean areTableServicesEnabled = writeConfig.areTableServicesEnabled(); - boolean areAsyncTableServicesEnabled = writeConfig.areAnyTableServicesAsync(); - if (!isLockProviderPropertySet && areTableServicesEnabled && areAsyncTableServicesEnabled) { - // This is targeted at Single writer with async table services - // If user does not set the lock provider, likely that the concurrency mode is not set either - // Override the configs for metadata table - writeConfig.setValue(WRITE_CONCURRENCY_MODE.key(), - WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value()); - writeConfig.setValue(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), - InProcessLockProvider.class.getName()); - LOG.info(String.format("Automatically set %s=%s and %s=%s since user has not set the " - + "lock provider for single writer with async table services", - WRITE_CONCURRENCY_MODE.key(), WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value(), - HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getName())); - } - } + // This is targeted at Single writer with async table services + // If user does not set the lock provider, likely that the concurrency mode is not set either + // Override the configs for metadata table + writeConfig.setValue(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), + InProcessLockProvider.class.getName()); + LOG.info(String.format("Automatically set %s=%s since user has not set the " + + "lock provider for single writer with async table services", + HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.key(), InProcessLockProvider.class.getName())); } // We check if "hoodie.cleaner.policy.failed.writes" @@ -2937,9 +3006,14 @@ private void validate() { // Ensure Layout Version is good new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); Objects.requireNonNull(writeConfig.getString(BASE_PATH)); + if (writeConfig.isEarlyConflictDetectionEnable()) { + checkArgument(writeConfig.getString(WRITE_CONCURRENCY_MODE) + .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value()), + "To use early conflict detection, set hoodie.write.concurrency.mode=OPTIMISTIC_CONCURRENCY_CONTROL"); + } if (writeConfig.getString(WRITE_CONCURRENCY_MODE) .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.value())) { - ValidationUtils.checkArgument(!writeConfig.getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY) + checkArgument(!writeConfig.getString(HoodieCleanConfig.FAILED_WRITES_CLEANER_POLICY) .equals(HoodieFailedWritesCleaningPolicy.EAGER.name()), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY"); } @@ -2950,12 +3024,12 @@ private void validate() { int maxInstantsToKeep = Integer.parseInt(writeConfig.getStringOrDefault(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP)); int cleanerCommitsRetained = Integer.parseInt(writeConfig.getStringOrDefault(HoodieCleanConfig.CLEANER_COMMITS_RETAINED)); - ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep, + checkArgument(maxInstantsToKeep > minInstantsToKeep, String.format( "Increase %s=%d to be greater than %s=%d.", HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), maxInstantsToKeep, HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), minInstantsToKeep)); - ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained, + checkArgument(minInstantsToKeep > cleanerCommitsRetained, String.format( "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull " + "missing data from few instants.", @@ -2964,14 +3038,21 @@ private void validate() { boolean inlineCompact = writeConfig.getBoolean(HoodieCompactionConfig.INLINE_COMPACT); boolean inlineCompactSchedule = writeConfig.getBoolean(HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT); - ValidationUtils.checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " + checkArgument(!(inlineCompact && inlineCompactSchedule), String.format("Either of inline compaction (%s) or " + "schedule inline compaction (%s) can be enabled. Both can't be set to true at the same time. %s, %s", HoodieCompactionConfig.INLINE_COMPACT.key(), HoodieCompactionConfig.SCHEDULE_INLINE_COMPACT.key(), inlineCompact, inlineCompactSchedule)); } public HoodieWriteConfig build() { + return build(true); + } + + @VisibleForTesting + public HoodieWriteConfig build(boolean shouldValidate) { setDefaults(); - validate(); + if (shouldValidate) { + validate(); + } // Build WriteConfig at the end return new HoodieWriteConfig(engineType, writeConfig.getProps()); } @@ -2982,7 +3063,9 @@ private String getDefaultMarkersType(EngineType engineType) { if (writeConfig.isEmbeddedTimelineServerEnabled()) { return MarkerType.TIMELINE_SERVER_BASED.toString(); } else { - LOG.warn("Embedded timeline server is disabled, fallback to use direct marker type for spark"); + if (!HoodieTableMetadata.isMetadataTable(writeConfig.getBasePath())) { + LOG.warn("Embedded timeline server is disabled, fallback to use direct marker type for spark"); + } return MarkerType.DIRECT.toString(); } case FLINK: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java index 20f75f63c5234..8cc8c07a02ec2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java @@ -21,13 +21,13 @@ import org.apache.avro.Schema; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.LazyIterableIterator; -import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.CreateHandleFactory; import org.apache.hudi.io.WriteHandleFactory; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.util.ExecutorFactory; import java.util.Iterator; import java.util.List; @@ -90,23 +90,25 @@ public R getResult() { } } - static Function, HoodieInsertValueGenResult> getCloningTransformer(Schema schema, - HoodieWriteConfig config) { - return getCloningTransformerInternal(schema, config.getProps()); + static Function, HoodieInsertValueGenResult> getTransformer(Schema schema, + HoodieWriteConfig writeConfig) { + return getTransformerInternal(schema, writeConfig); } - static Function, HoodieInsertValueGenResult> getCloningTransformer(Schema schema) { - return getCloningTransformerInternal(schema, new TypedProperties()); - } + private static Function, HoodieInsertValueGenResult> getTransformerInternal(Schema schema, + HoodieWriteConfig writeConfig) { + // NOTE: Whether record have to be cloned here is determined based on the executor type used + // for writing: executors relying on an inner queue, will be keeping references to the records + // and therefore in the environments where underlying buffer holding the record could be + // reused (for ex, Spark) we need to make sure that we get a clean copy of + // it since these records will be subsequently buffered (w/in the in-memory queue); + // Only case when we don't need to make a copy is when using [[SimpleExecutor]] which + // is guaranteed to not hold on to references to any records + boolean shouldClone = ExecutorFactory.isBufferingRecords(writeConfig); - private static Function, HoodieInsertValueGenResult> getCloningTransformerInternal(Schema schema, - TypedProperties props) { return record -> { - // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific - // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of - // it since these records will be subsequently buffered (w/in the in-memory queue) - HoodieRecord clonedRecord = record.copy(); - return new HoodieInsertValueGenResult(clonedRecord, schema, props); + HoodieRecord clonedRecord = shouldClone ? record.copy() : record; + return new HoodieInsertValueGenResult(clonedRecord, schema, writeConfig.getProps()); }; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index 8a8a03e1b17ac..2ec46f19a43cc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -62,9 +62,8 @@ public class HoodieIndexUtils { * @param hoodieTable Instance of {@link HoodieTable} of interest * @return the list of {@link HoodieBaseFile} */ - public static List getLatestBaseFilesForPartition( - final String partition, - final HoodieTable hoodieTable) { + public static List getLatestBaseFilesForPartition(String partition, + HoodieTable hoodieTable) { Option latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (latestCommitTime.isPresent()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java index 9430d9bb5e50b..f144540ed2280 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BaseHoodieBloomIndexHelper.java @@ -19,12 +19,11 @@ package org.apache.hudi.index.bloom; -import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -51,7 +50,7 @@ public abstract class BaseHoodieBloomIndexHelper implements Serializable { public abstract HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodiePairData fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index 57d9def9b4201..833db3b1c75a1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -35,7 +36,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.MetadataNotFoundException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; @@ -45,9 +45,9 @@ import org.apache.log4j.Logger; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import java.util.stream.Stream; import static java.util.stream.Collectors.groupingBy; @@ -91,7 +91,8 @@ record -> new ImmutablePair<>(record.getPartitionPath(), record.getRecordKey())) // Cache the result, for subsequent stages. if (config.getBloomIndexUseCaching()) { - keyFilenamePairs.persist("MEMORY_AND_DISK_SER"); + keyFilenamePairs.persist(new HoodieConfig(config.getProps()) + .getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE)); } if (LOG.isDebugEnabled()) { long totalTaggedRecords = keyFilenamePairs.count(); @@ -127,7 +128,7 @@ private HoodiePairData lookupIndex( // Step 3: Obtain a HoodieData, for each incoming record, that already exists, with the file id, // that contains it. - HoodieData> fileComparisonPairs = + HoodiePairData fileComparisonPairs = explodeRecordsWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairs); return bloomIndexHelper.findMatchingFilesForRecordKeys(config, context, hoodieTable, @@ -210,34 +211,35 @@ protected List> loadColumnRangesFromMetaIndex( // also obtain file ranges, if range pruning is enabled context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices: " + config.getTableName()); - final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); - return context.flatMap(partitions, partitionName -> { - // Partition and file name pairs - List> partitionFileNameList = HoodieIndexUtils.getLatestBaseFilesForPartition(partitionName, - hoodieTable).stream().map(baseFile -> Pair.of(partitionName, baseFile.getFileName())) - .sorted() - .collect(toList()); - if (partitionFileNameList.isEmpty()) { - return Stream.empty(); - } - try { - Map, HoodieMetadataColumnStats> fileToColumnStatsMap = - hoodieTable.getMetadataTable().getColumnStats(partitionFileNameList, keyField); - List> result = new ArrayList<>(); - for (Map.Entry, HoodieMetadataColumnStats> entry : fileToColumnStatsMap.entrySet()) { - result.add(Pair.of(entry.getKey().getLeft(), - new BloomIndexFileInfo( - FSUtils.getFileId(entry.getKey().getRight()), - // NOTE: Here we assume that the type of the primary key field is string - (String) unwrapStatisticValueWrapper(entry.getValue().getMinValue()), - (String) unwrapStatisticValueWrapper(entry.getValue().getMaxValue()) - ))); - } - return result.stream(); - } catch (MetadataNotFoundException me) { - throw new HoodieMetadataException("Unable to find column range metadata for partition:" + partitionName, me); - } - }, Math.max(partitions.size(), 1)); + String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + + // Partition and file name pairs + List> partitionFileNameList = + HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream() + .map(partitionBaseFilePair -> Pair.of(partitionBaseFilePair.getLeft(), partitionBaseFilePair.getRight().getFileName())) + .sorted() + .collect(toList()); + + if (partitionFileNameList.isEmpty()) { + return Collections.emptyList(); + } + + Map, HoodieMetadataColumnStats> fileToColumnStatsMap = + hoodieTable.getMetadataTable().getColumnStats(partitionFileNameList, keyField); + + List> result = new ArrayList<>(fileToColumnStatsMap.size()); + + for (Map.Entry, HoodieMetadataColumnStats> entry : fileToColumnStatsMap.entrySet()) { + result.add(Pair.of(entry.getKey().getLeft(), + new BloomIndexFileInfo( + FSUtils.getFileId(entry.getKey().getRight()), + // NOTE: Here we assume that the type of the primary key field is string + (String) unwrapStatisticValueWrapper(entry.getValue().getMinValue()), + (String) unwrapStatisticValueWrapper(entry.getValue().getMaxValue()) + ))); + } + + return result; } @Override @@ -278,7 +280,7 @@ public boolean isImplicitWithStorage() { * Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on * recordKey ranges in the index info. */ - HoodieData> explodeRecordsWithFileComparisons( + HoodiePairData explodeRecordsWithFileComparisons( final Map> partitionToFileIndexInfo, HoodiePairData partitionRecordKeyPairs) { IndexFileFilter indexFileFilter = @@ -289,11 +291,13 @@ HoodieData> explodeRecordsWithFileComparisons( String recordKey = partitionRecordKeyPair.getRight(); String partitionPath = partitionRecordKeyPair.getLeft(); - return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> (Pair) new ImmutablePair<>(partitionFileIdPair.getRight(), - new HoodieKey(recordKey, partitionPath))) - .collect(Collectors.toList()); - }).flatMap(List::iterator); + return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey) + .stream() + .map(partitionFileIdPair -> + new ImmutablePair<>( + new HoodieFileGroupId(partitionFileIdPair.getLeft(), partitionFileIdPair.getRight()), recordKey)); + }) + .flatMapToPair(Stream::iterator); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java similarity index 61% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java index 80031f4e8f025..52b504e9ab168 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBaseBloomIndexCheckFunction.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java @@ -19,7 +19,8 @@ package org.apache.hudi.index.bloom; import org.apache.hudi.client.utils.LazyIterableIterator; -import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.function.SerializableFunction; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -28,53 +29,67 @@ import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; -import java.util.function.Function; +import java.io.Serializable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.function.Function; /** - * Function performing actual checking of list containing (fileId, hoodieKeys) against the actual files. + * Function accepting a tuple of {@link HoodieFileGroupId} and a record key and producing + * a list of {@link HoodieKeyLookupResult} for every file identified by the file-group ids + * + * @param type of the tuple of {@code (HoodieFileGroupId, )}. Note that this is + * parameterized as generic such that this code could be reused for Spark as well */ -public class HoodieBaseBloomIndexCheckFunction - implements Function>, Iterator>> { +public class HoodieBloomIndexCheckFunction + implements Function, Iterator>>, Serializable { private final HoodieTable hoodieTable; private final HoodieWriteConfig config; - public HoodieBaseBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) { + private final SerializableFunction fileGroupIdExtractor; + private final SerializableFunction recordKeyExtractor; + + public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, + HoodieWriteConfig config, + SerializableFunction fileGroupIdExtractor, + SerializableFunction recordKeyExtractor) { this.hoodieTable = hoodieTable; this.config = config; + this.fileGroupIdExtractor = fileGroupIdExtractor; + this.recordKeyExtractor = recordKeyExtractor; } @Override - public Iterator> apply(Iterator> filePartitionRecordKeyTripletItr) { - return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); + public Iterator> apply(Iterator fileGroupIdRecordKeyPairIterator) { + return new LazyKeyCheckIterator(fileGroupIdRecordKeyPairIterator); } - class LazyKeyCheckIterator extends LazyIterableIterator, List> { + protected class LazyKeyCheckIterator extends LazyIterableIterator> { private HoodieKeyLookupHandle keyLookupHandle; - LazyKeyCheckIterator(Iterator> filePartitionRecordKeyTripletItr) { + LazyKeyCheckIterator(Iterator filePartitionRecordKeyTripletItr) { super(filePartitionRecordKeyTripletItr); } - @Override - protected void start() { - } - @Override protected List computeNext() { + List ret = new ArrayList<>(); try { // process one file in each go. while (inputItr.hasNext()) { - Pair currentTuple = inputItr.next(); - String fileId = currentTuple.getLeft(); - String partitionPath = currentTuple.getRight().getPartitionPath(); - String recordKey = currentTuple.getRight().getRecordKey(); + I tuple = inputItr.next(); + + HoodieFileGroupId fileGroupId = fileGroupIdExtractor.apply(tuple); + String recordKey = recordKeyExtractor.apply(tuple); + + String fileId = fileGroupId.getFileId(); + String partitionPath = fileGroupId.getPartitionPath(); + Pair partitionPathFilePair = Pair.of(partitionPath, fileId); // lazily init state @@ -100,15 +115,13 @@ protected List computeNext() { } } catch (Throwable e) { if (e instanceof HoodieException) { - throw e; + throw (HoodieException) e; } + throw new HoodieIndexException("Error checking bloom filter index. ", e); } - return ret; - } - @Override - protected void end() { + return ret; } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java index 5f2007ea53668..b5604312d3f7b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java @@ -25,6 +25,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; @@ -41,7 +42,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; +import java.util.stream.Stream; /** * This filter will only work with hoodie table since it will only load partitions @@ -74,7 +75,7 @@ List> loadColumnRangesFromFiles(List pa */ @Override - HoodieData> explodeRecordsWithFileComparisons( + HoodiePairData explodeRecordsWithFileComparisons( final Map> partitionToFileIndexInfo, HoodiePairData partitionRecordKeyPairs) { @@ -87,10 +88,11 @@ HoodieData> explodeRecordsWithFileComparisons( String partitionPath = partitionRecordKeyPair.getLeft(); return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream() - .map(partitionFileIdPair -> (Pair) new ImmutablePair<>(partitionFileIdPair.getRight(), - new HoodieKey(recordKey, partitionFileIdPair.getLeft()))) - .collect(Collectors.toList()); - }).flatMap(List::iterator); + .map(partitionFileIdPair -> + new ImmutablePair<>( + new HoodieFileGroupId(partitionFileIdPair.getLeft(), partitionFileIdPair.getRight()), recordKey)); + }) + .flatMapToPair(Stream::iterator); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java index cffee5ee74081..b47f5cf066c54 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedHoodieBloomIndexHelper.java @@ -19,20 +19,20 @@ package org.apache.hudi.index.bloom; -import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; -import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Map; @@ -56,21 +56,21 @@ public static ListBasedHoodieBloomIndexHelper getInstance() { public HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodiePairData fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition) { - List> fileComparisonPairList = + List> fileComparisonPairList = fileComparisonPairs.collectAsList().stream() .sorted(Comparator.comparing(Pair::getLeft)).collect(toList()); - List keyLookupResults = new ArrayList<>(); - Iterator> iterator = new HoodieBaseBloomIndexCheckFunction( - hoodieTable, config).apply(fileComparisonPairList.iterator()); - while (iterator.hasNext()) { - keyLookupResults.addAll(iterator.next()); - } + List keyLookupResults = + CollectionUtils.toStream( + new HoodieBloomIndexCheckFunction>(hoodieTable, config, Pair::getLeft, Pair::getRight) + .apply(fileComparisonPairList.iterator()) + ) + .flatMap(Collection::stream) + .filter(lr -> lr.getMatchingRecordKeys().size() > 0) + .collect(toList()); - keyLookupResults = keyLookupResults.stream().filter( - lr -> lr.getMatchingRecordKeys().size() > 0).collect(toList()); return context.parallelize(keyLookupResults).flatMap(lookupResult -> lookupResult.getMatchingRecordKeys().stream() .map(recordKey -> new ImmutablePair<>(lookupResult, recordKey)).iterator() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java index 35f9205a8e58b..5264c8b39f192 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/BucketIdentifier.java @@ -76,7 +76,11 @@ public static String partitionBucketIdStr(String partition, int bucketId) { } public static int bucketIdFromFileId(String fileId) { - return Integer.parseInt(fileId.substring(0, 8)); + return Integer.parseInt(bucketIdStrFromFileId(fileId)); + } + + public static String bucketIdStrFromFileId(String fileId) { + return fileId.substring(0, 8); } public static String bucketIdStr(int n) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 02053127d9c83..a69d460334362 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -114,9 +114,9 @@ public class HoodieAppendHandle extends HoodieWriteHandle header = new HashMap<>(); private SizeEstimator sizeEstimator; @@ -236,21 +236,25 @@ private Option prepareRecord(HoodieRecord hoodieRecord) { // If the format can not record the operation field, nullify the DELETE payload manually. boolean nullifyPayload = HoodieOperation.isDelete(hoodieRecord.getOperation()) && !config.allowOperationMetadataField(); recordProperties.put(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, String.valueOf(isUpdateRecord)); - Option finalRecord = nullifyPayload ? Option.empty() : Option.of(hoodieRecord); + + Option finalRecordOpt = nullifyPayload ? Option.empty() : Option.of(hoodieRecord); // Check for delete - if (finalRecord.isPresent() && !finalRecord.get().isDelete(schema, recordProperties)) { - // Check for ignore ExpressionPayload - if (finalRecord.get().shouldIgnore(schema, recordProperties)) { - return finalRecord; + if (finalRecordOpt.isPresent() && !finalRecordOpt.get().isDelete(schema, recordProperties)) { + HoodieRecord finalRecord = finalRecordOpt.get(); + // Check if the record should be ignored (special case for [[ExpressionPayload]]) + if (finalRecord.shouldIgnore(schema, recordProperties)) { + return finalRecordOpt; } - // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema - HoodieRecord rewrittenRecord = schemaOnReadEnabled ? finalRecord.get().rewriteRecordWithNewSchema(schema, recordProperties, writeSchemaWithMetaFields) - : finalRecord.get().rewriteRecord(schema, recordProperties, writeSchemaWithMetaFields); + + // Prepend meta-fields into the record + MetadataValues metadataValues = populateMetadataFields(finalRecord); + HoodieRecord populatedRecord = + finalRecord.prependMetaFields(schema, writeSchemaWithMetaFields, metadataValues, recordProperties); + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of // it since these records will be put into the recordList(List). - HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord.copy(), writeSchemaWithMetaFields, recordProperties); - finalRecord = Option.of(populatedRecord); + finalRecordOpt = Option.of(populatedRecord.copy()); if (isUpdateRecord || isLogCompaction) { updatedRecordsWritten++; } else { @@ -258,7 +262,7 @@ private Option prepareRecord(HoodieRecord hoodieRecord) { } recordsWritten++; } else { - finalRecord = Option.empty(); + finalRecordOpt = Option.empty(); recordsDeleted++; } @@ -267,7 +271,7 @@ private Option prepareRecord(HoodieRecord hoodieRecord) { // part of marking // record successful. hoodieRecord.deflate(); - return finalRecord; + return finalRecordOpt; } catch (Exception e) { LOG.error("Error writing record " + hoodieRecord, e); writeStatus.markFailure(hoodieRecord, e, recordMetadata); @@ -275,7 +279,7 @@ private Option prepareRecord(HoodieRecord hoodieRecord) { return Option.empty(); } - private HoodieRecord populateMetadataFields(HoodieRecord hoodieRecord, Schema schema, Properties prop) throws IOException { + private MetadataValues populateMetadataFields(HoodieRecord hoodieRecord) { MetadataValues metadataValues = new MetadataValues(); if (config.populateMetaFields()) { String seqId = @@ -292,7 +296,7 @@ private HoodieRecord populateMetadataFields(HoodieRecord hoodieRecord, Schema metadataValues.setOperation(hoodieRecord.getOperation().getName()); } - return hoodieRecord.updateMetadataValues(schema, prop, metadataValues); + return metadataValues; } private void initNewStatus() { @@ -588,7 +592,7 @@ private void flushToDiskIfRequired(HoodieRecord record, boolean appendDeleteBloc } // Append if max number of records reached to achieve block size - if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) { + if (numberOfRecords >= (long) (maxBlockSize / averageRecordSize)) { // Recompute averageRecordSize before writing a new block and update existing value with // avg of new and old LOG.info("Flush log block to disk, the current avgRecordSize => " + averageRecordSize); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java index f110bf585dbe3..e4985907e2e38 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -18,13 +18,19 @@ package org.apache.hudi.io; -import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.avro.JsonProperties; +import org.apache.avro.Schema; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema; + /** * This class is essentially same as Create Handle but overrides two things * 1) Schema : Metadata bootstrap writes only metadata fields as part of write. So, setup the writer schema accordingly. @@ -34,14 +40,28 @@ */ public class HoodieBootstrapHandle extends HoodieCreateHandle { + // NOTE: We have to use schema containing all the meta-fields in here b/c unlike for [[HoodieAvroRecord]], + // [[HoodieSparkRecord]] requires records to always bear either all or no meta-fields in the + // record schema (ie partial inclusion of the meta-fields in the schema is not allowed) + public static final Schema METADATA_BOOTSTRAP_RECORD_SCHEMA = createMetadataBootstrapRecordSchema(); + public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) { super(config, commitTime, hoodieTable, partitionPath, fileId, - Option.of(HoodieAvroUtils.RECORD_KEY_SCHEMA), taskContextSupplier); + Option.of(METADATA_BOOTSTRAP_RECORD_SCHEMA), taskContextSupplier); } @Override public boolean canWrite(HoodieRecord record) { return true; } + + private static Schema createMetadataBootstrapRecordSchema() { + List fields = + HoodieRecord.HOODIE_META_COLUMNS.stream() + .map(metaField -> + new Schema.Field(metaField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE)) + .collect(Collectors.toList()); + return Schema.createRecord("HoodieRecordKey", "", "", false, fields); + } } \ No newline at end of file diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java index be646df85be4d..d80f4250c60bd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCDCLogger.java @@ -58,6 +58,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before_after; + /** * This class encapsulates all the cdc-writing functions. */ @@ -89,7 +92,7 @@ public class HoodieCDCLogger implements Closeable { private final CDCTransformer transformer; // Max block size to limit to for a log block - private final int maxBlockSize; + private final long maxBlockSize; // Average cdc record size. This size is updated at the end of every log block flushed to disk private long averageCDCRecordSize = 0; @@ -240,10 +243,10 @@ public void close() { // ------------------------------------------------------------------------- private CDCTransformer getTransformer() { - if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) { + if (cdcSupplementalLoggingMode == data_before_after) { return (operation, recordKey, oldRecord, newRecord) -> - HoodieCDCUtils.cdcRecord(cdcSchema, operation.getValue(), commitTime, removeCommitMetadata(oldRecord), newRecord); - } else if (cdcSupplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE)) { + HoodieCDCUtils.cdcRecord(cdcSchema, operation.getValue(), commitTime, removeCommitMetadata(oldRecord), removeCommitMetadata(newRecord)); + } else if (cdcSupplementalLoggingMode == data_before) { return (operation, recordKey, oldRecord, newRecord) -> HoodieCDCUtils.cdcRecord(cdcSchema, operation.getValue(), recordKey, removeCommitMetadata(oldRecord)); } else { 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 96611959f9a5a..4d3f52a5ba6aa 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 @@ -136,24 +136,22 @@ protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props if (record.shouldIgnore(schema, config.getProps())) { return; } - // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema - HoodieRecord rewriteRecord; - if (schemaOnReadEnabled) { - rewriteRecord = record.rewriteRecordWithNewSchema(schema, config.getProps(), writeSchemaWithMetaFields); - } else { - rewriteRecord = record.rewriteRecord(schema, config.getProps(), writeSchemaWithMetaFields); - } + MetadataValues metadataValues = new MetadataValues().setFileName(path.getName()); - rewriteRecord = rewriteRecord.updateMetadataValues(writeSchemaWithMetaFields, config.getProps(), metadataValues); + HoodieRecord populatedRecord = + record.prependMetaFields(schema, writeSchemaWithMetaFields, metadataValues, config.getProps()); + if (preserveMetadata) { - fileWriter.write(record.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields); + fileWriter.write(record.getRecordKey(), populatedRecord, writeSchemaWithMetaFields); } else { - fileWriter.writeWithMetadata(record.getKey(), rewriteRecord, writeSchemaWithMetaFields); + fileWriter.writeWithMetadata(record.getKey(), populatedRecord, writeSchemaWithMetaFields); } - // update the new location of record, so we know where to find it next + + // Update the new location of record, so we know where to find it next record.unseal(); record.setNewLocation(new HoodieRecordLocation(instantTime, writeStatus.getFileId())); record.seal(); + recordsWritten++; insertRecordsWritten++; } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java index 0460f88101c54..f92a1e73d5849 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java @@ -374,20 +374,16 @@ public void write(HoodieRecord oldRecord) { } protected void writeToFile(HoodieKey key, HoodieRecord record, Schema schema, Properties prop, boolean shouldPreserveRecordMetadata) throws IOException { - HoodieRecord rewriteRecord; - if (schemaOnReadEnabled) { - rewriteRecord = record.rewriteRecordWithNewSchema(schema, prop, writeSchemaWithMetaFields); - } else { - rewriteRecord = record.rewriteRecord(schema, prop, writeSchemaWithMetaFields); - } // NOTE: `FILENAME_METADATA_FIELD` has to be rewritten to correctly point to the // file holding this record even in cases when overall metadata is preserved MetadataValues metadataValues = new MetadataValues().setFileName(newFilePath.getName()); - rewriteRecord = rewriteRecord.updateMetadataValues(writeSchemaWithMetaFields, prop, metadataValues); + HoodieRecord populatedRecord = + record.prependMetaFields(schema, writeSchemaWithMetaFields, metadataValues, prop); + if (shouldPreserveRecordMetadata) { - fileWriter.write(key.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields); + fileWriter.write(key.getRecordKey(), populatedRecord, writeSchemaWithMetaFields); } else { - fileWriter.writeWithMetadata(key, rewriteRecord, writeSchemaWithMetaFields); + fileWriter.writeWithMetadata(key, populatedRecord, writeSchemaWithMetaFields); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java index b110c2c081782..8aadd637f0e52 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java @@ -80,21 +80,11 @@ public static HoodieMergeHandle create( TaskContextSupplier taskContextSupplier, Option keyGeneratorOpt) { if (table.requireSortedRecords()) { - if (table.getMetaClient().getTableConfig().isCDCEnabled()) { - return new HoodieSortedMergeHandleWithChangeLog<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); - } else { - return new HoodieSortedMergeHandle<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); - } + return new HoodieSortedMergeHandle<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } else { - if (table.getMetaClient().getTableConfig().isCDCEnabled()) { - return new HoodieMergeHandleWithChangeLog<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); - } else { - return new HoodieMergeHandle<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, - dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); - } + return new HoodieMergeHandle<>(writeConfig, instantTime, table, keyToNewRecords, partitionPath, fileId, + dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 8e470471db91a..889d7a64769a4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -95,7 +95,7 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); this.taskContextSupplier = taskContextSupplier; this.writeToken = makeWriteToken(); - schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema()); + this.schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema()); this.recordMerger = config.getRecordMerger(); } 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 bbfa4460af411..78c4b1fc44192 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 @@ -26,6 +26,7 @@ import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.BaseHoodieWriteClient; +import org.apache.hudi.client.FailOnFirstErrorWriteStatus; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.data.HoodieData; @@ -70,7 +71,7 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.hadoop.CachingPath; import org.apache.hudi.hadoop.SerializablePath; - +import org.apache.hudi.table.action.compact.strategy.UnBoundedCompactionStrategy; import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -93,6 +94,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.getIndexInflightInstant; @@ -100,6 +102,7 @@ import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.METADATA_INDEXER_TIME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; @@ -134,15 +137,17 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta /** * Hudi backed table metadata writer. * - * @param hadoopConf - Hadoop configuration to use for the metadata writer - * @param writeConfig - Writer config - * @param engineContext - Engine context - * @param actionMetadata - Optional action metadata to help decide initialize operations - * @param - Action metadata types extending Avro generated SpecificRecordBase - * @param inflightInstantTimestamp - Timestamp of any instant in progress + * @param hadoopConf Hadoop configuration to use for the metadata writer + * @param writeConfig Writer config + * @param failedWritesCleaningPolicy Cleaning policy on failed writes + * @param engineContext Engine context + * @param actionMetadata Optional action metadata to help decide initialize operations + * @param Action metadata types extending Avro generated SpecificRecordBase + * @param inflightInstantTimestamp Timestamp of any instant in progress */ protected HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option actionMetadata, Option inflightInstantTimestamp) { @@ -154,7 +159,7 @@ protected HoodieBackedTableMetadataWriter(Configu if (writeConfig.isMetadataTableEnabled()) { this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX; - this.metadataWriteConfig = createMetadataWriteConfig(writeConfig); + this.metadataWriteConfig = createMetadataWriteConfig(writeConfig, failedWritesCleaningPolicy); enabled = true; // Inline compaction and auto clean is required as we dont expose this table outside @@ -162,9 +167,12 @@ protected HoodieBackedTableMetadataWriter(Configu "Cleaning is controlled internally for Metadata table."); ValidationUtils.checkArgument(!this.metadataWriteConfig.inlineCompactionEnabled(), "Compaction is controlled internally for metadata table."); - // Metadata Table cannot have metadata listing turned on. (infinite loop, much?) + // Auto commit is required ValidationUtils.checkArgument(this.metadataWriteConfig.shouldAutoCommit(), "Auto commit is required for Metadata Table"); + ValidationUtils.checkArgument(this.metadataWriteConfig.getWriteStatusClassName().equals(FailOnFirstErrorWriteStatus.class.getName()), + "MDT should use " + FailOnFirstErrorWriteStatus.class.getName()); + // Metadata Table cannot have metadata listing turned on. (infinite loop, much?) ValidationUtils.checkArgument(!this.metadataWriteConfig.isMetadataTableEnabled(), "File listing cannot be used for Metadata Table"); @@ -181,7 +189,7 @@ protected HoodieBackedTableMetadataWriter(Configu public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) { - this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty()); + this(hadoopConf, writeConfig, EAGER, engineContext, Option.empty(), Option.empty()); } /** @@ -232,11 +240,14 @@ private void enablePartition(final MetadataPartitionType partitionType, final Ho protected abstract void initRegistry(); /** - * Create a {@code HoodieWriteConfig} to use for the Metadata Table. + * Create a {@code HoodieWriteConfig} to use for the Metadata Table. This is used by async + * indexer only. * - * @param writeConfig {@code HoodieWriteConfig} of the main dataset writer + * @param writeConfig {@code HoodieWriteConfig} of the main dataset writer + * @param failedWritesCleaningPolicy Cleaning policy on failed writes */ - private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfig) { + private HoodieWriteConfig createMetadataWriteConfig( + HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy) { int parallelism = writeConfig.getMetadataInsertParallelism(); int minCommitsToKeep = Math.max(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMinCommitsToKeep()); @@ -268,7 +279,7 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withAutoClean(false) .withCleanerParallelism(parallelism) .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS) - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER) + .withFailedWritesCleaningPolicy(failedWritesCleaningPolicy) .retainCommits(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED.defaultValue()) .build()) // we will trigger archive manually, to ensure only regular writer invokes it @@ -283,7 +294,11 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi // by default, the HFile does not keep the metadata fields, set up as false // to always use the metadata of the new record. .withPreserveCommitMetadata(false) - .withLogRecordReaderScanV2(String.valueOf(writeConfig.useScanV2ForLogRecordReader())) + .withEnableOptimizedLogBlocksScan(String.valueOf(writeConfig.enableOptimizedLogBlocksScan())) + // Compaction on metadata table is used as a barrier for archiving on main dataset and for validating the + // deltacommits having corresponding completed commits. Therefore, we need to compact all fileslices of all + // partitions together requiring UnBoundedCompactionStrategy. + .withCompactionStrategy(new UnBoundedCompactionStrategy()) .build()) .withParallelism(parallelism, parallelism) .withDeleteParallelism(parallelism) @@ -291,7 +306,10 @@ private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfi .withFinalizeWriteParallelism(parallelism) .withAllowMultiWriteOnSameInstant(true) .withKeyGenerator(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) - .withPopulateMetaFields(HoodieMetadataConfig.POPULATE_META_FIELDS.defaultValue()); + .withPopulateMetaFields(HoodieMetadataConfig.POPULATE_META_FIELDS.defaultValue()) + .withWriteStatusClass(FailOnFirstErrorWriteStatus.class) + .withReleaseResourceEnabled(writeConfig.areReleaseResourceEnabled()); + // RecordKey properties are needed for the metadata table records final Properties properties = new Properties(); @@ -648,12 +666,9 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC for (DirectoryInfo dirInfo : processedDirectories) { if (!dirFilterRegex.isEmpty()) { final String relativePath = dirInfo.getRelativePath(); - if (!relativePath.isEmpty()) { - Path partitionPath = new Path(datasetBasePath, relativePath); - if (partitionPath.getName().matches(dirFilterRegex)) { - LOG.info("Ignoring directory " + partitionPath + " which matches the filter regex " + dirFilterRegex); - continue; - } + if (!relativePath.isEmpty() && relativePath.matches(dirFilterRegex)) { + LOG.info("Ignoring directory " + relativePath + " which matches the filter regex " + dirFilterRegex); + continue; } } @@ -875,7 +890,7 @@ public void buildMetadataPartitions(HoodieEngineContext engineContext, List 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(latestDeltaCommitTimeInMetadataTable).getInstants(); + .findInstantsBeforeOrEquals(latestDeltaCommitTimeInMetadataTable).getInstants(); if (!pendingInstants.isEmpty()) { LOG.info(String.format( @@ -1069,6 +1091,7 @@ protected void cleanIfNecessary(BaseHoodieWriteClient writeClient, String instan // delta commits synced over will not have an instant time lesser than the last completed instant on the // metadata table. writeClient.clean(instantTime + "002"); + writeClient.lazyRollbackFailedIndexing(); } /** @@ -1086,39 +1109,45 @@ private void initialCommit(String createInstantTime, List Map> partitionToRecordsMap = new HashMap<>(); - List partitionInfoList = listAllPartitions(dataMetaClient); - Map> partitionToFilesMap = partitionInfoList.stream() - .map(p -> { - String partitionName = HoodieTableMetadataUtil.getPartitionIdentifier(p.getRelativePath()); - return Pair.of(partitionName, p.getFileNameToSizeMap()); - }) - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - - int totalDataFilesCount = partitionToFilesMap.values().stream().mapToInt(Map::size).sum(); - List partitions = new ArrayList<>(partitionToFilesMap.keySet()); - - if (partitionTypes.contains(MetadataPartitionType.FILES)) { - // Record which saves the list of all partitions - HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions); - HoodieData filesPartitionRecords = getFilesPartitionRecords(createInstantTime, partitionInfoList, allPartitionRecord); - ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); - partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); - } + // skip file system listing to populate metadata records if its a fresh table. + // this is applicable only if the table already has N commits and metadata is enabled at a later point in time. + if (createInstantTime.equals(SOLO_COMMIT_TIMESTAMP)) { // SOLO_COMMIT_TIMESTAMP will be the initial commit time in MDT for a fresh table. + // If not, last completed commit in data table will be chosen as the initial commit time. + LOG.info("Triggering empty Commit to metadata to initialize"); + } else { + List partitionInfoList = listAllPartitions(dataMetaClient); + Map> partitionToFilesMap = partitionInfoList.stream() + .map(p -> { + String partitionName = HoodieTableMetadataUtil.getPartitionIdentifier(p.getRelativePath()); + return Pair.of(partitionName, p.getFileNameToSizeMap()); + }) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + + int totalDataFilesCount = partitionToFilesMap.values().stream().mapToInt(Map::size).sum(); + List partitions = new ArrayList<>(partitionToFilesMap.keySet()); + + if (partitionTypes.contains(MetadataPartitionType.FILES)) { + // Record which saves the list of all partitions + HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions); + HoodieData filesPartitionRecords = getFilesPartitionRecords(createInstantTime, partitionInfoList, allPartitionRecord); + ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); + } - if (partitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS) && totalDataFilesCount > 0) { - final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( - engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams(), createInstantTime); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, recordsRDD); - } + if (partitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS) && totalDataFilesCount > 0) { + final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( + engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams(), createInstantTime); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, recordsRDD); + } - if (partitionTypes.contains(MetadataPartitionType.COLUMN_STATS) && totalDataFilesCount > 0) { - final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( - engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams()); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, recordsRDD); + if (partitionTypes.contains(MetadataPartitionType.COLUMN_STATS) && totalDataFilesCount > 0) { + final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( + engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams()); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, recordsRDD); + } + LOG.info("Committing " + partitions.size() + " partitions and " + totalDataFilesCount + " files to metadata"); } - LOG.info("Committing " + partitions.size() + " partitions and " + totalDataFilesCount + " files to metadata"); - commit(createInstantTime, partitionToRecordsMap, false); } 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 af8d8d2326187..9800cf268ac4e 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 @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; @@ -41,6 +42,7 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FailSafeConsistencyGuard; import org.apache.hudi.common.fs.OptimisticConsistencyGuard; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieTableType; @@ -91,6 +93,10 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -99,7 +105,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.avro.AvroSchemaUtils.isSchemaCompatible; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.LAZY; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; @@ -654,23 +661,20 @@ public void finalizeWrite(HoodieEngineContext context, String instantTs, List>> invalidFilesByPartition) { // Now delete partially written files context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation: " + config.getTableName()); - context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> { - final FileSystem fileSystem = metaClient.getFs(); - LOG.info("Deleting invalid data files=" + partitionWithFileList); - if (partitionWithFileList.isEmpty()) { - return true; - } - // Delete - partitionWithFileList.stream().map(Pair::getValue).forEach(file -> { - try { - fileSystem.delete(new Path(file), false); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - }); - - return true; - }, config.getFinalizeWriteParallelism()); + context.map(invalidFilesByPartition.values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()), + partitionFilePair -> { + final FileSystem fileSystem = metaClient.getFs(); + LOG.info("Deleting invalid data file=" + partitionFilePair); + // Delete + try { + fileSystem.delete(new Path(partitionFilePair.getValue()), false); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + return true; + }, config.getFinalizeWriteParallelism()); } /** @@ -800,27 +804,22 @@ public TaskContextSupplier getTaskContextSupplier() { */ private void validateSchema() throws HoodieUpsertException, HoodieInsertException { - if (!config.shouldValidateAvroSchema() || getActiveTimeline().getCommitsTimeline().filterCompletedInstants().empty()) { + boolean shouldValidate = config.shouldValidateAvroSchema(); + boolean allowProjection = config.shouldAllowAutoEvolutionColumnDrop(); + if ((!shouldValidate && allowProjection) + || getActiveTimeline().getCommitsTimeline().filterCompletedInstants().empty()) { // Check not required return; } - Schema tableSchema; - Schema writerSchema; - boolean isValid; try { TableSchemaResolver schemaResolver = new TableSchemaResolver(getMetaClient()); - writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema()); - tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaResolver.getTableAvroSchemaWithoutMetadataFields()); - isValid = isSchemaCompatible(tableSchema, writerSchema); + Schema writerSchema = HoodieAvroUtils.createHoodieWriteSchema(config.getSchema()); + Schema tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaResolver.getTableAvroSchema(false)); + AvroSchemaUtils.checkSchemaCompatible(tableSchema, writerSchema, shouldValidate, allowProjection, getDropPartitionColNames()); } catch (Exception e) { throw new HoodieException("Failed to read schema/check compatibility for base path " + metaClient.getBasePath(), e); } - - if (!isValid) { - throw new HoodieException("Failed schema compatibility check for writerSchema :" + writerSchema - + ", table schema :" + tableSchema + ", base path :" + metaClient.getBasePath()); - } } public void validateUpsertSchema() throws HoodieUpsertException { @@ -872,7 +871,8 @@ public HoodieEngineContext getContext() { * @return instance of {@link HoodieTableMetadataWriter} */ public final Option getMetadataWriter(String triggeringInstantTimestamp) { - return getMetadataWriter(triggeringInstantTimestamp, Option.empty()); + return getMetadataWriter( + triggeringInstantTimestamp, EAGER, Option.empty()); } /** @@ -895,6 +895,29 @@ public boolean isTableServiceAction(String actionType, String instantTime) { } } + /** + * Gets the metadata writer for async indexer. + * + * @param triggeringInstantTimestamp The instant that is triggering this metadata write. + * @return An instance of {@link HoodieTableMetadataWriter}. + */ + public Option getIndexingMetadataWriter(String triggeringInstantTimestamp) { + return getMetadataWriter(triggeringInstantTimestamp, LAZY, Option.empty()); + } + + /** + * Gets the metadata writer for regular writes. + * + * @param triggeringInstantTimestamp The instant that is triggering this metadata write. + * @param actionMetadata Optional action metadata. + * @param Action metadata type. + * @return An instance of {@link HoodieTableMetadataWriter}. + */ + public Option getMetadataWriter( + String triggeringInstantTimestamp, Option actionMetadata) { + return getMetadataWriter(triggeringInstantTimestamp, EAGER, actionMetadata); + } + /** * Get Table metadata writer. *

@@ -905,11 +928,14 @@ public boolean isTableServiceAction(String actionType, String instantTime) { * are blocked from doing the similar initial metadata table creation and * the bootstrapping. * - * @param triggeringInstantTimestamp - The instant that is triggering this metadata write + * @param triggeringInstantTimestamp The instant that is triggering this metadata write + * @param failedWritesCleaningPolicy Cleaning policy on failed writes * @return instance of {@link HoodieTableMetadataWriter} */ - public Option getMetadataWriter(String triggeringInstantTimestamp, - Option actionMetadata) { + protected Option getMetadataWriter( + String triggeringInstantTimestamp, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + Option actionMetadata) { // Each engine is expected to override this and // provide the actual metadata writer, if enabled. return Option.empty(); @@ -1010,4 +1036,16 @@ public HoodieTableMetadata getMetadataTable() { public Runnable getPreExecuteRunnable() { return Functions.noop(); } + + private Set getDropPartitionColNames() { + boolean shouldDropPartitionColumns = metaClient.getTableConfig().shouldDropPartitionColumns(); + if (!shouldDropPartitionColumns) { + return Collections.emptySet(); + } + Option partitionFields = metaClient.getTableConfig().getPartitionFields(); + if (!partitionFields.isPresent()) { + return Collections.emptySet(); + } + return new HashSet<>(Arrays.asList(partitionFields.get())); + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java index 9137eb436bb8a..01b8d19122651 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java @@ -126,11 +126,11 @@ private static Stream> deleteFilesFunc(Iterator */ List clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) { int cleanerParallelism = Math.min( - (int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()), + cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum(), config.getCleanerParallelism()); LOG.info("Using cleanerParallelism: " + cleanerParallelism); - context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions: " + config.getTableName()); + context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of table: " + config.getTableName()); Stream> filesToBeDeletedPerPartition = cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream() diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java index 982800cc2463f..1db4d6ae96cd8 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 @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.CleanFileInfo; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; @@ -29,10 +30,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; -import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.cdc.HoodieCDCUtils; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -45,7 +43,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieSavepointException; -import org.apache.hudi.metadata.FileSystemBackedTableMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; @@ -54,7 +51,6 @@ import java.io.IOException; import java.io.Serializable; import java.time.Instant; -import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Collections; @@ -62,7 +58,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -166,7 +161,8 @@ private List getPartitionPathsForCleanByCommits(Option in HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils .deserializeHoodieCleanMetadata(hoodieTable.getActiveTimeline().getInstantDetails(lastClean.get()).get()); if ((cleanMetadata.getEarliestCommitToRetain() != null) - && (cleanMetadata.getEarliestCommitToRetain().length() > 0)) { + && (cleanMetadata.getEarliestCommitToRetain().length() > 0) + && !hoodieTable.getActiveTimeline().isBeforeTimelineStarts(cleanMetadata.getEarliestCommitToRetain())) { return getPartitionPathsForIncrementalCleaning(cleanMetadata, instantToRetain); } } @@ -213,15 +209,7 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata */ private List getPartitionPathsForFullCleaning() { // Go to brute force mode of scanning all partitions - try { - // Because the partition of BaseTableMetadata has been deleted, - // all partition information can only be obtained from FileSystemBackedTableMetadata. - FileSystemBackedTableMetadata fsBackedTableMetadata = new FileSystemBackedTableMetadata(context, - context.getHadoopConf(), config.getBasePath(), config.shouldAssumeDatePartitioning()); - return fsBackedTableMetadata.getAllPartitionPaths(); - } catch (IOException e) { - return Collections.emptyList(); - } + return FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), config.getBasePath()); } /** @@ -367,13 +355,10 @@ private Pair> getFilesToCleanKeepingLatestCommits(S deletePaths.add(new CleanFileInfo(hoodieDataFile.getBootstrapBaseFile().get().getPath(), true)); } }); - if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ - || hoodieTable.getMetaClient().getTableConfig().isCDCEnabled()) { - // 1. If merge on read, then clean the log files for the commits as well; - // 2. If change log capture is enabled, clean the log files no matter the table type is mor or cow. - deletePaths.addAll(aSlice.getLogFiles().map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) - .collect(Collectors.toList())); - } + // clean the log files for the commits, which contain cdc log files in cdc scenario + // and normal log files for mor tables. + deletePaths.addAll(aSlice.getLogFiles().map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) + .collect(Collectors.toList())); } } } @@ -436,23 +421,12 @@ private List getCleanFileInfoForSlice(FileSlice nextSlice) { cleanPaths.add(new CleanFileInfo(dataFile.getBootstrapBaseFile().get().getPath(), true)); } } - if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { - // If merge on read, then clean the log files for the commits as well - Predicate notCDCLogFile = - hoodieLogFile -> !hoodieLogFile.getFileName().endsWith(HoodieCDCUtils.CDC_LOGFILE_SUFFIX); - cleanPaths.addAll( - nextSlice.getLogFiles().filter(notCDCLogFile).map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) - .collect(Collectors.toList())); - } - if (hoodieTable.getMetaClient().getTableConfig().isCDCEnabled()) { - // The cdc log files will be written out in cdc scenario, no matter the table type is mor or cow. - // Here we need to clean uo these cdc log files. - Predicate isCDCLogFile = - hoodieLogFile -> hoodieLogFile.getFileName().endsWith(HoodieCDCUtils.CDC_LOGFILE_SUFFIX); - cleanPaths.addAll( - nextSlice.getLogFiles().filter(isCDCLogFile).map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) - .collect(Collectors.toList())); - } + + // clean the log files for the commits, which contain cdc log files in cdc scenario + // and normal log files for mor tables. + cleanPaths.addAll( + nextSlice.getLogFiles().map(lf -> new CleanFileInfo(lf.getPath().toString(), false)) + .collect(Collectors.toList())); return cleanPaths; } @@ -507,7 +481,7 @@ public Option getEarliestCommitToRetain() { } } else if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) { Instant instant = Instant.now(); - ZonedDateTime currentDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); + ZonedDateTime currentDateTime = ZonedDateTime.ofInstant(instant, hoodieTable.getMetaClient().getTableConfig().getTimelineTimezone().getZoneId()); String earliestTimeToRetain = HoodieActiveTimeline.formatDate(Date.from(currentDateTime.minusHours(hoursRetained).toInstant())); earliestCommitToRetain = Option.fromJavaOptional(commitTimeline.getInstantsAsStream().filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestTimeToRetain)).findFirst()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 1e92f80227482..10be7e7be06ac 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -42,6 +42,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieClusteringException; @@ -62,6 +63,7 @@ import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -69,6 +71,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; + public abstract class BaseCommitActionExecutor extends BaseActionExecutor { @@ -77,9 +81,9 @@ public abstract class BaseCommitActionExecutor protected final Option> extraMetadata; protected final WriteOperationType operationType; protected final TaskContextSupplier taskContextSupplier; - protected final TransactionManager txnManager; - protected Option>> lastCompletedTxn; - protected Set pendingInflightAndRequestedInstants; + protected final Option txnManagerOption; + protected final Option>> lastCompletedTxn; + protected final Set pendingInflightAndRequestedInstants; public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType, @@ -89,11 +93,16 @@ public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig c this.extraMetadata = extraMetadata; this.taskContextSupplier = context.getTaskContextSupplier(); // TODO : Remove this once we refactor and move out autoCommit method from here, since the TxnManager is held in {@link BaseHoodieWriteClient}. - this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); - this.lastCompletedTxn = txnManager.isOptimisticConcurrencyControlEnabled() - ? TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()) : Option.empty(); - this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(table.getMetaClient()); - this.pendingInflightAndRequestedInstants.remove(instantTime); + this.txnManagerOption = config.shouldAutoCommit() ? Option.of(new TransactionManager(config, table.getMetaClient().getFs())) : Option.empty(); + if (this.txnManagerOption.isPresent() && this.txnManagerOption.get().isLockRequired()) { + // these txn metadata are only needed for auto commit when optimistic concurrent control is also enabled + this.lastCompletedTxn = TransactionUtils.getLastCompletedTxnInstantAndMetadata(table.getMetaClient()); + this.pendingInflightAndRequestedInstants = TransactionUtils.getInflightAndRequestedInstants(table.getMetaClient()); + this.pendingInflightAndRequestedInstants.remove(instantTime); + } else { + this.lastCompletedTxn = Option.empty(); + this.pendingInflightAndRequestedInstants = Collections.emptySet(); + } if (!table.getStorageLayout().writeOperationSupported(operationType)) { throw new UnsupportedOperationException("Executor " + this.getClass().getSimpleName() + " is not compatible with table layout " + table.getStorageLayout().getClass().getSimpleName()); @@ -180,16 +189,18 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { protected void autoCommit(Option> extraMetadata, HoodieWriteMetadata result) { final Option inflightInstant = Option.of(new HoodieInstant(State.INFLIGHT, getCommitActionType(), instantTime)); - this.txnManager.beginTransaction(inflightInstant, + ValidationUtils.checkState(this.txnManagerOption.isPresent(), "The transaction manager has not been initialized"); + TransactionManager txnManager = this.txnManagerOption.get(); + txnManager.beginTransaction(inflightInstant, lastCompletedTxn.isPresent() ? Option.of(lastCompletedTxn.get().getLeft()) : Option.empty()); try { setCommitMetadata(result); // reload active timeline so as to get all updates after current transaction have started. hence setting last arg to true. - TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(), - result.getCommitMetadata(), config, this.txnManager.getLastCompletedTransactionOwner(), true, pendingInflightAndRequestedInstants); + TransactionUtils.resolveWriteConflictIfAny(table, txnManager.getCurrentTransactionOwner(), + result.getCommitMetadata(), config, txnManager.getLastCompletedTransactionOwner(), true, pendingInflightAndRequestedInstants); commit(extraMetadata, result); } finally { - this.txnManager.endTransaction(inflightInstant); + txnManager.endTransaction(inflightInstant); } } @@ -246,6 +257,8 @@ protected HoodieWriteMetadata> executeClustering(HoodieC .performClustering(clusteringPlan, schema, instantTime); HoodieData writeStatusList = writeMetadata.getWriteStatuses(); HoodieData statuses = updateIndex(writeStatusList, writeMetadata); + statuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); + // triggers clustering. writeMetadata.setWriteStats(statuses.map(WriteStatus::getStat).collectAsList()); writeMetadata.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(clusteringPlan, writeMetadata)); commitOnAutoCommit(writeMetadata); @@ -290,5 +303,4 @@ private void validateWriteResult(HoodieClusteringPlan clusteringPlan, HoodieWrit + " write statuses"); } } - } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index 7f46e211e7391..3c8255a21b9a3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -27,8 +27,6 @@ import org.apache.hudi.common.util.ClosableIterator; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.MappingIterator; -import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -94,8 +92,8 @@ public void runMerge(HoodieTable table, // In case Advanced Schema Evolution is enabled we might need to rewrite currently // persisted records to adhere to an evolved schema - Option>, Schema>> schemaEvolutionTransformerOpt = - composeSchemaEvolutionTransformer(writerSchema, baseFile, writeConfig, table.getMetaClient()); + Option> schemaEvolutionTransformerOpt = + composeSchemaEvolutionTransformer(readerSchema, writerSchema, baseFile, writeConfig, table.getMetaClient()); // Check whether the writer schema is simply a projection of the file's one, ie // - Its field-set is a proper subset (of the reader schema) @@ -130,29 +128,27 @@ public void runMerge(HoodieTable table, (left, right) -> left.joinWith(right, mergeHandle.getWriterSchemaWithMetaFields())); recordSchema = mergeHandle.getWriterSchemaWithMetaFields(); - } else if (schemaEvolutionTransformerOpt.isPresent()) { - recordIterator = new MappingIterator<>(baseFileRecordIterator, - schemaEvolutionTransformerOpt.get().getLeft().apply(isPureProjection ? writerSchema : readerSchema)); - recordSchema = schemaEvolutionTransformerOpt.get().getRight(); } else { recordIterator = baseFileRecordIterator; recordSchema = isPureProjection ? writerSchema : readerSchema; } + boolean isBufferingRecords = ExecutorFactory.isBufferingRecords(writeConfig); + wrapper = ExecutorFactory.create(writeConfig, recordIterator, new UpdateHandler(mergeHandle), record -> { + HoodieRecord newRecord; + if (schemaEvolutionTransformerOpt.isPresent()) { + newRecord = schemaEvolutionTransformerOpt.get().apply(record); + } else if (shouldRewriteInWriterSchema) { + newRecord = record.rewriteRecordWithNewSchema(recordSchema, writeConfig.getProps(), writerSchema); + } else { + newRecord = record; + } + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of // it since these records will be put into queue of QueueBasedExecutorFactory. - if (shouldRewriteInWriterSchema) { - try { - return record.rewriteRecordWithNewSchema(recordSchema, writeConfig.getProps(), writerSchema).copy(); - } catch (IOException e) { - LOG.error("Error rewrite record with new schema", e); - throw new HoodieException(e); - } - } else { - return record.copy(); - } + return isBufferingRecords ? newRecord.copy() : newRecord; }, table.getPreExecuteRunnable()); wrapper.execute(); @@ -173,10 +169,11 @@ public void runMerge(HoodieTable table, } } - private Option>, Schema>> composeSchemaEvolutionTransformer(Schema writerSchema, - HoodieBaseFile baseFile, - HoodieWriteConfig writeConfig, - HoodieTableMetaClient metaClient) { + private Option> composeSchemaEvolutionTransformer(Schema recordSchema, + Schema writerSchema, + HoodieBaseFile baseFile, + HoodieWriteConfig writeConfig, + HoodieTableMetaClient metaClient) { Option querySchemaOpt = SerDeHelper.fromJson(writeConfig.getInternalSchema()); // TODO support bootstrap if (querySchemaOpt.isPresent() && !baseFile.getBootstrapBaseFile().isPresent()) { @@ -214,18 +211,12 @@ private Option>, Sche || SchemaCompatibility.checkReaderWriterCompatibility(newWriterSchema, writeSchemaFromFile).getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; if (needToReWriteRecord) { Map renameCols = InternalSchemaUtils.collectRenameCols(writeInternalSchema, querySchema); - return Option.of(Pair.of( - (schema) -> (record) -> { - try { - return record.rewriteRecordWithNewSchema( - schema, - writeConfig.getProps(), - newWriterSchema, renameCols); - } catch (IOException e) { - LOG.error("Error rewrite record with new schema", e); - throw new HoodieException(e); - } - }, newWriterSchema)); + return Option.of(record -> { + return record.rewriteRecordWithNewSchema( + recordSchema, + writeConfig.getProps(), + newWriterSchema, renameCols); + }); } else { return Option.empty(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java index b8e761acb02a9..47793d7155e12 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.util.collection.Pair; @@ -77,8 +78,10 @@ protected HoodieData> doDeduplicateRecords( } catch (IOException e) { throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e); } - HoodieKey reducedKey = rec1.getData().equals(reducedRecord.getData()) ? rec1.getKey() : rec2.getKey(); - return reducedRecord.newInstance(reducedKey); + boolean choosePrev = rec1.getData().equals(reducedRecord.getData()); + HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey(); + HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation(); + return reducedRecord.newInstance(reducedKey, operation); }, reduceParallelism).map(Pair::getRight); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java index f4498a82d4fb8..800e6a4aceac4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieCommitMetadata; 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.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -65,6 +66,7 @@ public HoodieCommitMetadata createCompactionMetadata( metadata.addWriteStat(stat.getPartitionPath(), stat); } metadata.addMetadata(org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY, schema); + metadata.setOperationType(WriteOperationType.COMPACT); if (compactionPlan.getExtraMetadata() != null) { compactionPlan.getExtraMetadata().forEach(metadata::addMetadata); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java index f402a673598dc..bdb836161967e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java @@ -71,8 +71,8 @@ protected Iterator> writeFileAndGetWriteStats(HoodieCompaction return result; } - protected boolean useScanV2(HoodieWriteConfig writeConfig) { - return writeConfig.useScanV2ForLogRecordReader(); + protected boolean enableOptimizedLogBlockScan(HoodieWriteConfig writeConfig) { + return writeConfig.enableOptimizedLogBlocksScan(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index c6a20436c03ef..0d18a68cbad49 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -199,7 +199,7 @@ public List compact(HoodieCompactionHandler compactionHandler, .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) .withOperationField(config.allowOperationMetadataField()) .withPartition(operation.getPartitionPath()) - .withUseScanV2(executionHelper.useScanV2(config)) + .withOptimizedLogBlocksScan(executionHelper.enableOptimizedLogBlockScan(config)) .withRecordMerger(config.getRecordMerger()) .build(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java index 0e49267507c2e..8d2b054d09fc6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java @@ -80,7 +80,7 @@ protected Iterator> writeFileAndGetWriteStats(HoodieCompaction } @Override - protected boolean useScanV2(HoodieWriteConfig writeConfig) { + protected boolean enableOptimizedLogBlockScan(HoodieWriteConfig writeConfig) { return true; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java index 6a5f160f6b09c..e7a77002cc5d5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/HoodieLogCompactionPlanGenerator.java @@ -90,7 +90,7 @@ private boolean isFileSliceEligibleForLogCompaction(FileSlice fileSlice, String .collect(Collectors.toList())) .withLatestInstantTime(maxInstantTime) .withBufferSize(writeConfig.getMaxDFSStreamBufferSize()) - .withUseScanV2(true) + .withOptimizedLogBlocksScan(true) .withRecordMerger(writeConfig.getRecordMerger()) .build(); scanner.scan(true); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 2fcbfb2b2e5b6..19aab3629d506 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -138,8 +138,9 @@ public Option execute() { List finalIndexPartitionInfos = null; if (!firstTimeInitializingMetadataTable) { // start indexing for each partition - HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) - .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); + HoodieTableMetadataWriter metadataWriter = table.getIndexingMetadataWriter(instantTime) + .orElseThrow(() -> new HoodieIndexException(String.format( + "Could not get metadata writer to run index action for instant: %s", instantTime))); // this will only build index upto base instant as generated by the plan, we will be doing catchup later String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); LOG.info("Starting Index Building with base instant: " + indexUptoInstant); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java index 000ea21af987e..45870501cab20 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java @@ -93,7 +93,7 @@ public List getRollbackRequests(HoodieInstant instantToRo // - Base file's file-id // - Base file's commit instant // - Partition path - return getRollbackRequestForAppend(WriteMarkers.stripMarkerSuffix(markerFilePath)); + return getRollbackRequestForAppend(instantToRollback, WriteMarkers.stripMarkerSuffix(markerFilePath)); default: throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback); } @@ -103,7 +103,7 @@ public List getRollbackRequests(HoodieInstant instantToRo } } - protected HoodieRollbackRequest getRollbackRequestForAppend(String markerFilePath) throws IOException { + protected HoodieRollbackRequest getRollbackRequestForAppend(HoodieInstant instantToRollback, String markerFilePath) throws IOException { Path baseFilePathForAppend = new Path(basePath, markerFilePath); String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend); String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName()); @@ -115,6 +115,14 @@ protected HoodieRollbackRequest getRollbackRequestForAppend(String markerFilePat // TODO(HUDI-1517) use provided marker-file's path instead Option latestLogFileOption = FSUtils.getLatestLogFile(table.getMetaClient().getFs(), partitionPath, fileId, HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime); + + // Log file can be deleted if the commit to rollback is also the commit that created the fileGroup + if (latestLogFileOption.isPresent() && baseCommitTime.equals(instantToRollback.getTimestamp())) { + Path fullDeletePath = new Path(partitionPath, latestLogFileOption.get().getFileName()); + return new HoodieRollbackRequest(relativePartitionPath, EMPTY_STRING, EMPTY_STRING, + Collections.singletonList(fullDeletePath.toString()), + Collections.emptyMap()); + } Map logFilesWithBlocsToRollback = new HashMap<>(); if (latestLogFileOption.isPresent()) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java index b9e7f06f84859..49e83733adf01 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/util/ExecutorFactory.java @@ -20,11 +20,11 @@ import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; -import org.apache.hudi.common.util.queue.SimpleHoodieExecutor; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.common.util.queue.ExecutorType; -import org.apache.hudi.common.util.queue.HoodieExecutor; import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.common.util.queue.HoodieExecutor; +import org.apache.hudi.common.util.queue.SimpleExecutor; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -33,29 +33,45 @@ public class ExecutorFactory { - public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, - Iterator inputItr, - HoodieConsumer consumer, - Function transformFunction) { - return create(hoodieConfig, inputItr, consumer, transformFunction, Functions.noop()); + public static HoodieExecutor create(HoodieWriteConfig config, + Iterator inputItr, + HoodieConsumer consumer, + Function transformFunction) { + return create(config, inputItr, consumer, transformFunction, Functions.noop()); } - public static HoodieExecutor create(HoodieWriteConfig hoodieConfig, - Iterator inputItr, - HoodieConsumer consumer, - Function transformFunction, - Runnable preExecuteRunnable) { - ExecutorType executorType = hoodieConfig.getExecutorType(); - + public static HoodieExecutor create(HoodieWriteConfig config, + Iterator inputItr, + HoodieConsumer consumer, + Function transformFunction, + Runnable preExecuteRunnable) { + ExecutorType executorType = config.getExecutorType(); switch (executorType) { case BOUNDED_IN_MEMORY: - return new BoundedInMemoryExecutor<>(hoodieConfig.getWriteBufferLimitBytes(), inputItr, consumer, + return new BoundedInMemoryExecutor<>(config.getWriteBufferLimitBytes(), inputItr, consumer, transformFunction, preExecuteRunnable); case DISRUPTOR: - return new DisruptorExecutor<>(hoodieConfig.getDisruptorWriteBufferSize(), inputItr, consumer, - transformFunction, hoodieConfig.getWriteExecutorWaitStrategy(), preExecuteRunnable); + return new DisruptorExecutor<>(config.getWriteExecutorDisruptorWriteBufferLimitBytes(), inputItr, consumer, + transformFunction, config.getWriteExecutorDisruptorWaitStrategy(), preExecuteRunnable); + case SIMPLE: + return new SimpleExecutor<>(inputItr, consumer, transformFunction); + default: + throw new HoodieException("Unsupported Executor Type " + executorType); + } + } + + /** + * Checks whether configured {@link HoodieExecutor} buffer records (for ex, by holding them + * in the queue) + */ + public static boolean isBufferingRecords(HoodieWriteConfig config) { + ExecutorType executorType = config.getExecutorType(); + switch (executorType) { + case BOUNDED_IN_MEMORY: + case DISRUPTOR: + return true; case SIMPLE: - return new SimpleHoodieExecutor<>(inputItr, consumer, transformFunction); + return false; default: throw new HoodieException("Unsupported Executor Type " + executorType); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java new file mode 100644 index 0000000000000..df5b03ec7dd45 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java @@ -0,0 +1,178 @@ +/* + * 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.transaction; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +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.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieWriteConflictException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class TestBucketIndexConcurrentFileWritesConflictResolutionStrategy extends HoodieCommonTestHarness { + + @BeforeEach + public void init() throws IOException { + initMetaClient(); + } + + @Test + public void testNoConcurrentWrites() throws Exception { + String newInstantTime = HoodieTestTable.makeNewCommitTime(); + createCommit(newInstantTime); + // consider commits before this are all successful + + Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); + newInstantTime = HoodieTestTable.makeNewCommitTime(); + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant); + Assertions.assertEquals(0, candidateInstants.count()); + } + + @Test + public void testConcurrentWrites() throws Exception { + String newInstantTime = HoodieTestTable.makeNewCommitTime(); + createCommit(newInstantTime); + // consider commits before this are all successful + // writer 1 + createInflightCommit(HoodieTestTable.makeNewCommitTime(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + // writer 2 + createInflightCommit(HoodieTestTable.makeNewCommitTime(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); + newInstantTime = HoodieTestTable.makeNewCommitTime(); + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + Stream candidateInstants = strategy.getCandidateInstants(metaClient.getActiveTimeline(), currentInstant.get(), lastSuccessfulInstant); + Assertions.assertEquals(0, candidateInstants.count()); + } + + @Test + public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + // writer 2 starts and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCommit(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + // writer 1 conflicts with writer 2 + Assertions.assertEquals(1, candidateInstants.size()); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + try { + strategy.resolveConflict(null, thisCommitOperation, thatCommitOperation); + Assertions.fail("Cannot reach here, writer 1 and writer 2 should have thrown a conflict"); + } catch (HoodieWriteConflictException e) { + // expected + } + } + + @Test + public void testConcurrentWritesWithDifferentPartition() throws Exception { + createCommit(HoodieActiveTimeline.createNewInstantTime()); + HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); + // consider commits before this are all successful + Option lastSuccessfulInstant = timeline.getCommitsTimeline().filterCompletedInstants().lastInstant(); + // writer 1 starts + String currentWriterInstant = HoodieActiveTimeline.createNewInstantTime(); + createInflightCommit(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH); + // writer 2 starts and finishes + String newInstantTime = HoodieActiveTimeline.createNewInstantTime(); + createCommit(newInstantTime); + + Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); + HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH); + timeline = timeline.reload(); + List candidateInstants = strategy.getCandidateInstants(timeline, currentInstant.get(), lastSuccessfulInstant).collect( + Collectors.toList()); + + // there should be 1 candidate instant + Assertions.assertEquals(1, candidateInstants.size()); + ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); + ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); + // there should be no conflict between writer 1 and writer 2 + Assertions.assertFalse(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); + } + + private void createCommit(String instantTime) throws Exception { + String fileId1 = "00000001-file-" + instantTime + "-1"; + String fileId2 = "00000002-file-" + instantTime + "-2"; + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(fileId1); + commitMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); + commitMetadata.setOperationType(WriteOperationType.INSERT); + HoodieTestTable.of(metaClient) + .addCommit(instantTime, Option.of(commitMetadata)) + .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } + + private HoodieCommitMetadata createCommitMetadata(String instantTime, String writeFileName, String partition) { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + commitMetadata.addMetadata("test", "test"); + HoodieWriteStat writeStat = new HoodieWriteStat(); + writeStat.setFileId(writeFileName); + commitMetadata.addWriteStat(partition, writeStat); + commitMetadata.setOperationType(WriteOperationType.INSERT); + return commitMetadata; + } + + private HoodieCommitMetadata createCommitMetadata(String instantTime, String partition) { + return createCommitMetadata(instantTime, "00000001-file-" + instantTime + "-1", partition); + } + + private void createInflightCommit(String instantTime, String partition) throws Exception { + String fileId1 = "00000001-file-" + instantTime + "-1"; + String fileId2 = "00000002-file-" + instantTime + "-2"; + HoodieTestTable.of(metaClient) + .addInflightCommit(instantTime) + .withBaseFilesInPartition(partition, fileId1, fileId2); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java index 99ab0887e7033..058785cd6a765 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestInProcessLockProvider.java @@ -26,9 +26,13 @@ import org.apache.hudi.exception.HoodieLockException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; + +import junit.framework.AssertionFailedError; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -51,6 +55,120 @@ public TestInProcessLockProvider() { lockConfiguration2 = new LockConfiguration(properties); } + @Test + public void testLockIdentity() throws InterruptedException { + // The lifecycle of an InProcessLockProvider should not affect the singleton lock + // for a single table, i.e., all three writers should hold the same underlying lock instance + // on the same table. + // Writer 1: lock |----------------| unlock and close + // Writer 2: try lock | ... lock |------| unlock and close + // Writer 3: try lock | ... lock |------| unlock and close + List lockProviderList = new ArrayList<>(); + InProcessLockProvider lockProvider1 = new InProcessLockProvider(lockConfiguration1, hadoopConfiguration); + lockProviderList.add(lockProvider1); + AtomicBoolean writer1Completed = new AtomicBoolean(false); + AtomicBoolean writer2TryLock = new AtomicBoolean(false); + AtomicBoolean writer2Locked = new AtomicBoolean(false); + AtomicBoolean writer2Completed = new AtomicBoolean(false); + AtomicBoolean writer3TryLock = new AtomicBoolean(false); + AtomicBoolean writer3Completed = new AtomicBoolean(false); + + // Writer 1 + assertDoesNotThrow(() -> { + LOG.info("Writer 1 tries to acquire the lock."); + lockProvider1.lock(); + LOG.info("Writer 1 acquires the lock."); + }); + // Writer 2 thread in parallel, should block + // and later acquire the lock once it is released + Thread writer2 = new Thread(() -> { + InProcessLockProvider lockProvider2 = new InProcessLockProvider(lockConfiguration1, hadoopConfiguration); + lockProviderList.add(lockProvider2); + assertDoesNotThrow(() -> { + LOG.info("Writer 2 tries to acquire the lock."); + writer2TryLock.set(true); + lockProvider2.lock(); + LOG.info("Writer 2 acquires the lock."); + }); + writer2Locked.set(true); + + while (!writer3TryLock.get()) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + + assertDoesNotThrow(() -> { + lockProvider2.unlock(); + LOG.info("Writer 2 releases the lock."); + }); + lockProvider2.close(); + LOG.info("Writer 2 closes the lock provider."); + writer2Completed.set(true); + }); + + Thread writer3 = new Thread(() -> { + while (!writer2Locked.get() || !writer1Completed.get()) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + // Lock instance of Writer 3 should be held by Writer 2 + InProcessLockProvider lockProvider3 = new InProcessLockProvider(lockConfiguration1, hadoopConfiguration); + lockProviderList.add(lockProvider3); + boolean isLocked = lockProvider3.getLock().isWriteLocked(); + if (!isLocked) { + writer3TryLock.set(true); + throw new AssertionFailedError("The lock instance in Writer 3 should be held by Writer 2: " + + lockProvider3.getLock()); + } + assertDoesNotThrow(() -> { + LOG.info("Writer 3 tries to acquire the lock."); + writer3TryLock.set(true); + lockProvider3.lock(); + LOG.info("Writer 3 acquires the lock."); + }); + + assertDoesNotThrow(() -> { + lockProvider3.unlock(); + LOG.info("Writer 3 releases the lock."); + }); + lockProvider3.close(); + LOG.info("Writer 3 closes the lock provider."); + writer3Completed.set(true); + }); + + writer2.start(); + writer3.start(); + + while (!writer2TryLock.get()) { + Thread.sleep(100); + } + + assertDoesNotThrow(() -> { + lockProvider1.unlock(); + LOG.info("Writer 1 releases the lock."); + lockProvider1.close(); + LOG.info("Writer 1 closes the lock provider."); + writer1Completed.set(true); + }); + + try { + writer2.join(); + writer3.join(); + } catch (InterruptedException e) { + // Ignore any exception + } + Assertions.assertTrue(writer2Completed.get()); + Assertions.assertTrue(writer3Completed.get()); + Assertions.assertEquals(lockProviderList.get(0).getLock(), lockProviderList.get(1).getLock()); + Assertions.assertEquals(lockProviderList.get(1).getLock(), lockProviderList.get(2).getLock()); + } + @Test public void testLockAcquisition() { InProcessLockProvider inProcessLockProvider = new InProcessLockProvider(lockConfiguration1, hadoopConfiguration); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestLockManager.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestLockManager.java new file mode 100644 index 0000000000000..5e528f33f7431 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestLockManager.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.transaction; + +import org.apache.hudi.client.transaction.lock.LockManager; +import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.config.HoodieCleanConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.curator.test.TestingServer; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mockito; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +public class TestLockManager extends HoodieCommonTestHarness { + + private static final Logger LOG = LogManager.getLogger(TestLockManager.class); + + private static TestingServer server; + private static final String ZK_BASE_PATH = "/hudi/test/lock"; + private static final String KEY = "table1"; + + @BeforeAll + public static void setup() { + while (server == null) { + try { + server = new TestingServer(); + } catch (Exception e) { + LOG.error("Getting bind exception - retrying to allocate server"); + server = null; + } + } + } + + @AfterAll + public static void tearDown() throws IOException { + if (server != null) { + server.close(); + } + } + + @BeforeEach + void init() throws IOException { + initPath(); + initMetaClient(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testLockAndUnlock(boolean multiWriter) { + HoodieWriteConfig writeConfig = multiWriter ? getMultiWriterWriteConfig() : getSingleWriterWriteConfig(); + LockManager lockManager = new LockManager(writeConfig, this.metaClient.getFs()); + LockManager mockLockManager = Mockito.spy(lockManager); + + assertDoesNotThrow(() -> { + mockLockManager.lock(); + }); + + assertDoesNotThrow(() -> { + mockLockManager.unlock(); + }); + + Mockito.verify(mockLockManager).close(); + } + + private HoodieWriteConfig getMultiWriterWriteConfig() { + return HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withCleanConfig(HoodieCleanConfig.newBuilder() + .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY) + .build()) + .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(ZookeeperBasedLockProvider.class) + .withZkBasePath(ZK_BASE_PATH) + .withZkLockKey(KEY) + .withZkQuorum(server.getConnectString()) + .build()) + .build(); + } + + private HoodieWriteConfig getSingleWriterWriteConfig() { + return HoodieWriteConfig.newBuilder() + .withPath(basePath) + .withLockConfig(HoodieLockConfig.newBuilder() + .withLockProvider(ZookeeperBasedLockProvider.class) + .withZkBasePath(ZK_BASE_PATH) + .withZkLockKey(KEY) + .withZkQuorum(server.getConnectString()) + .build()) + .build(); + } +} 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..3a1632737efa9 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java @@ -0,0 +1,110 @@ +/* + * 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 = 3; + 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 hasPendingLogCompactionOperations, + boolean hasFileGroupsInPendingClustering) { + System.out.printf("hasPendingCompactionOperations: %s, hasPendingLogCompactionOperations: %s, hasFileGroupsInPendingClustering: %s%n", + hasPendingCompactionOperations, hasPendingLogCompactionOperations, hasFileGroupsInPendingClustering); + Mockito.when(table.getSliceView()).thenReturn(fileSystemView); + Mockito.when(fileSystemView.getPendingCompactionOperations()).thenReturn(createPendingCompactionOperations(hasPendingCompactionOperations)); + Mockito.when(fileSystemView.getPendingLogCompactionOperations()).thenReturn(createPendingCompactionOperations(hasPendingLogCompactionOperations)); + Mockito.when(fileSystemView.getFileGroupsInPendingClustering()).thenReturn(createFileGroupsInPendingClustering(hasFileGroupsInPendingClustering)); + + boolean shouldThrowException = hasPendingCompactionOperations || hasPendingLogCompactionOperations || 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/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java index e7afa50a59d88..1f4d6b1858727 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java @@ -140,8 +140,10 @@ public void testAutoConcurrencyConfigAdjustmentWithTableServices(HoodieTableType put(ASYNC_CLEAN.key(), "false"); put(HoodieWriteConfig.AUTO_ADJUST_LOCK_CONFIGS.key(), "true"); } - }), true, true, true, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, - HoodieFailedWritesCleaningPolicy.LAZY, inProcessLockProviderClassName); + }), true, true, true, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + inProcessLockProviderClassName); // 2. Async clean verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { @@ -153,8 +155,10 @@ public void testAutoConcurrencyConfigAdjustmentWithTableServices(HoodieTableType put(ASYNC_CLEAN.key(), "true"); put(HoodieWriteConfig.AUTO_ADJUST_LOCK_CONFIGS.key(), "true"); } - }), true, true, true, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, - HoodieFailedWritesCleaningPolicy.LAZY, inProcessLockProviderClassName); + }), true, true, true, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + inProcessLockProviderClassName); // 3. Async compaction configured verifyConcurrencyControlRelatedConfigs(createWriteConfig(new HashMap() { @@ -168,12 +172,8 @@ public void testAutoConcurrencyConfigAdjustmentWithTableServices(HoodieTableType } }), true, tableType == HoodieTableType.MERGE_ON_READ, true, - tableType == HoodieTableType.MERGE_ON_READ - ? WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL - : WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), - tableType == HoodieTableType.MERGE_ON_READ - ? HoodieFailedWritesCleaningPolicy.LAZY - : HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), tableType == HoodieTableType.MERGE_ON_READ ? inProcessLockProviderClassName : HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME.defaultValue()); @@ -205,8 +205,9 @@ public void testAutoConcurrencyConfigAdjustmentWithTableServices(HoodieTableType put(HoodieWriteConfig.AUTO_ADJUST_LOCK_CONFIGS.key(), "true"); } }), true, true, false, - WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, - HoodieFailedWritesCleaningPolicy.LAZY, inProcessLockProviderClassName); + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), + inProcessLockProviderClassName); } @ParameterizedTest @@ -289,8 +290,8 @@ public void testAutoConcurrencyConfigAdjustmentWithUserConfigs(HoodieTableType t if (writeConfig.areAnyTableServicesAsync()) { verifyConcurrencyControlRelatedConfigs(writeConfig, true, true, true, - WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL, - HoodieFailedWritesCleaningPolicy.LAZY, + WriteConcurrencyMode.valueOf(WRITE_CONCURRENCY_MODE.defaultValue()), + HoodieFailedWritesCleaningPolicy.valueOf(FAILED_WRITES_CLEANER_POLICY.defaultValue()), InProcessLockProvider.class.getName()); } else { verifyConcurrencyControlRelatedConfigs(writeConfig, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index 815b04c9dbc87..70d2ebfec034a 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -112,7 +112,7 @@ protected HoodieAvroHFileWriter createWriter( protected HoodieAvroFileReader createReader( Configuration conf) throws Exception { CacheConfig cacheConfig = new CacheConfig(conf); - return new HoodieAvroHFileReader(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf)); + return new HoodieAvroHFileReader(conf, getFilePath(), cacheConfig, getFilePath().getFileSystem(conf), Option.empty()); } @Override diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml index e3ddc8b9ed326..e530ffa6692b5 100644 --- a/hudi-client/hudi-flink-client/pom.xml +++ b/hudi-client/hudi-flink-client/pom.xml @@ -20,12 +20,12 @@ hudi-client org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink-client - 0.13.0-SNAPSHOT + 0.13.1 hudi-flink-client jar @@ -261,27 +261,6 @@ org.jacoco jacoco-maven-plugin - - net.alchim31.maven - scala-maven-plugin - - - scala-compile-first - process-resources - - add-source - compile - - - - scala-test-compile - process-test-resources - - testCompile - - - - org.apache.maven.plugins maven-compiler-plugin diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java index 4b3eaaa1d42cd..fea590122b8eb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java @@ -19,6 +19,7 @@ package org.apache.hudi.client; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieKey; @@ -62,8 +63,10 @@ public class HoodieFlinkTableServiceClient extends BaseHoodieTableServiceClie */ private HoodieBackedTableMetadataWriter metadataWriter; - protected HoodieFlinkTableServiceClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig); + protected HoodieFlinkTableServiceClient(HoodieEngineContext context, + HoodieWriteConfig clientConfig, + Option timelineService) { + super(context, clientConfig, timelineService); } @Override 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 7d0728b95eb42..96e08528dc5a1 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 @@ -86,7 +86,7 @@ public class HoodieFlinkWriteClient extends public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) { super(context, writeConfig, FlinkUpgradeDowngradeHelper.getInstance()); this.bucketToHandles = new HashMap<>(); - this.tableServiceClient = new HoodieFlinkTableServiceClient<>(context, writeConfig); + this.tableServiceClient = new HoodieFlinkTableServiceClient<>(context, writeConfig, getTimelineServer()); } /** diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java index 6979716624910..6e573ec9432b6 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/execution/FlinkLazyInsertIterable.java @@ -60,7 +60,7 @@ protected List computeNext() { try { final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); bufferedIteratorExecutor = ExecutorFactory.create(hoodieConfig, inputItr, getExplicitInsertHandler(), - getCloningTransformer(schema, hoodieConfig)); + getTransformer(schema, hoodieConfig)); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); return result; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java index b939498c3e240..4a3109db60a33 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java @@ -20,11 +20,11 @@ import org.apache.hudi.avro.HoodieBloomFilterWriteSupport; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.util.Option; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.common.util.Option; import org.apache.parquet.hadoop.api.WriteSupport; import java.nio.charset.StandardCharsets; diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java index 3d9524eaa30e9..e5b9509d8798a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetRowDataWriter.java @@ -18,6 +18,8 @@ package org.apache.hudi.io.storage.row.parquet; +import org.apache.hudi.common.util.ValidationUtils; + import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalDataUtils; import org.apache.flink.table.data.MapData; @@ -124,17 +126,19 @@ private FieldWriter createWriter(LogicalType t) { return new DoubleWriter(); case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) t; - if (timestampType.getPrecision() == 3) { - return new Timestamp64Writer(); + final int tsPrecision = timestampType.getPrecision(); + if (tsPrecision == 3 || tsPrecision == 6) { + return new Timestamp64Writer(tsPrecision); } else { - return new Timestamp96Writer(timestampType.getPrecision()); + return new Timestamp96Writer(tsPrecision); } case TIMESTAMP_WITH_LOCAL_TIME_ZONE: LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t; - if (localZonedTimestampType.getPrecision() == 3) { - return new Timestamp64Writer(); + final int tsLtzPrecision = localZonedTimestampType.getPrecision(); + if (tsLtzPrecision == 3 || tsLtzPrecision == 6) { + return new Timestamp64Writer(tsLtzPrecision); } else { - return new Timestamp96Writer(localZonedTimestampType.getPrecision()); + return new Timestamp96Writer(tsLtzPrecision); } case ARRAY: ArrayType arrayType = (ArrayType) t; @@ -284,33 +288,64 @@ public void write(ArrayData array, int ordinal) { } /** - * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See + * TIMESTAMP_MILLIS and TIMESTAMP_MICROS is the deprecated ConvertedType of TIMESTAMP with the MILLIS and MICROS + * precision respectively. See * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp - * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. */ private class Timestamp64Writer implements FieldWriter { - private Timestamp64Writer() { + private final int precision; + private Timestamp64Writer(int precision) { + ValidationUtils.checkArgument(precision == 3 || precision == 6, + "Timestamp64Writer is only able to support precisions of {3, 6}"); + this.precision = precision; } @Override public void write(RowData row, int ordinal) { - recordConsumer.addLong(timestampToInt64(row.getTimestamp(ordinal, 3))); + TimestampData timestampData = row.getTimestamp(ordinal, precision); + recordConsumer.addLong(timestampToInt64(timestampData, precision)); } @Override public void write(ArrayData array, int ordinal) { - recordConsumer.addLong(timestampToInt64(array.getTimestamp(ordinal, 3))); + TimestampData timestampData = array.getTimestamp(ordinal, precision); + recordConsumer.addLong(timestampToInt64(timestampData, precision)); } } - private long timestampToInt64(TimestampData timestampData) { - return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime(); + /** + * Converts a {@code TimestampData} to its corresponding int64 value. This function only accepts TimestampData of + * precision 3 or 6. Special attention will need to be given to a TimestampData of precision = 6. + *

+ * For example representing `1970-01-01T00:00:03.100001` of precision 6 will have: + *

    + *
  • millisecond = 3100
  • + *
  • nanoOfMillisecond = 1000
  • + *
+ * As such, the int64 value will be: + *

+ * millisecond * 1000 + nanoOfMillisecond / 1000 + * + * @param timestampData TimestampData to be converted to int64 format + * @param precision the precision of the TimestampData + * @return int64 value of the TimestampData + */ + private long timestampToInt64(TimestampData timestampData, int precision) { + if (precision == 3) { + return utcTimestamp ? timestampData.getMillisecond() : timestampData.toTimestamp().getTime(); + } else { + // using an else clause here as precision has been validated to be {3, 6} in the constructor + // convert timestampData to microseconds format + return utcTimestamp ? timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000 : + timestampData.toTimestamp().getTime() * 1000; + } } /** * Timestamp of INT96 bytes, julianDay(4) + nanosOfDay(8). See * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp - * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. + *

+ * TODO: Leaving this here as there might be a requirement to support TIMESTAMP(9) in the future */ private class Timestamp96Writer implements FieldWriter { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java index 5fb76f9418c86..8bd1a8488553d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/parquet/ParquetSchemaConverter.java @@ -600,9 +600,10 @@ private static Type convertToParquetType( .named(name); case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) type; - if (timestampType.getPrecision() == 3) { + if (timestampType.getPrecision() == 3 || timestampType.getPrecision() == 6) { + TimeUnit timeunit = timestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS; return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS)) + .as(LogicalTypeAnnotation.timestampType(true, timeunit)) .named(name); } else { return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition) @@ -610,9 +611,10 @@ private static Type convertToParquetType( } case TIMESTAMP_WITH_LOCAL_TIME_ZONE: LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) type; - if (localZonedTimestampType.getPrecision() == 3) { + if (localZonedTimestampType.getPrecision() == 3 || localZonedTimestampType.getPrecision() == 6) { + TimeUnit timeunit = localZonedTimestampType.getPrecision() == 3 ? TimeUnit.MILLIS : TimeUnit.MICROS; return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MILLIS)) + .as(LogicalTypeAnnotation.timestampType(false, timeunit)) .named(name); } else { return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition) @@ -648,7 +650,7 @@ private static Type convertToParquetType( .addField( Types .repeatedGroup() - .addField(convertToParquetType("key", keyType, repetition)) + .addField(convertToParquetType("key", keyType, Type.Repetition.REQUIRED)) .addField(convertToParquetType("value", valueType, repetition)) .named("key_value")) .named(name); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index aa70f5835c8aa..7142db7084faa 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -23,13 +23,13 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.metrics.Registry; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.avro.specific.SpecificRecordBase; @@ -42,6 +42,8 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; + /** * Flink hoodie backed table metadata writer. */ @@ -58,7 +60,7 @@ public static HoodieTableMetadataWriter create(Co HoodieWriteConfig writeConfig, HoodieEngineContext context, Option actionMetadata) { - return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, Option.empty()); + return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, EAGER, context, actionMetadata, Option.empty()); } public static HoodieTableMetadataWriter create(Configuration conf, @@ -66,15 +68,27 @@ public static HoodieTableMetadataWriter create(Co HoodieEngineContext context, Option actionMetadata, Option inFlightInstantTimestamp) { - return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, inFlightInstantTimestamp); + return new FlinkHoodieBackedTableMetadataWriter( + conf, writeConfig, EAGER, context, actionMetadata, inFlightInstantTimestamp); + } + + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext context, + Option actionMetadata, + Option inFlightInstantTimestamp) { + return new FlinkHoodieBackedTableMetadataWriter( + conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inFlightInstantTimestamp); } FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option actionMetadata, Option inFlightInstantTimestamp) { - super(hadoopConf, writeConfig, engineContext, actionMetadata, inFlightInstantTimestamp); + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inFlightInstantTimestamp); } @Override @@ -150,11 +164,6 @@ protected void commit(String instantTime, Map statuses = preppedRecordList.size() > 0 ? writeClient.upsertPreppedRecords(preppedRecordList, instantTime) : Collections.emptyList(); - statuses.forEach(writeStatus -> { - if (writeStatus.hasErrors()) { - throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); - } - }); // flink does not support auto-commit yet, also the auto commit logic is not complete as BaseHoodieWriteClient now. writeClient.commit(instantTime, statuses, Option.empty(), HoodieActiveTimeline.DELTA_COMMIT_ACTION, Collections.emptyMap()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 3d77844df6f8b..422fe310b0c03 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -22,6 +22,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -98,11 +99,14 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - public Option getMetadataWriter(String triggeringInstantTimestamp, - Option actionMetadata) { + protected Option getMetadataWriter( + String triggeringInstantTimestamp, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + Option actionMetadata) { if (config.isMetadataTableEnabled()) { - return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, - context, actionMetadata, Option.of(triggeringInstantTimestamp))); + return Option.of(FlinkHoodieBackedTableMetadataWriter.create( + context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context, + actionMetadata, Option.of(triggeringInstantTimestamp))); } else { return Option.empty(); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java index a301ba228e4f6..3f19534d08cdb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java @@ -21,6 +21,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.engine.HoodieEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -62,6 +63,8 @@ public FlinkDeletePartitionCommitActionExecutor(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-flink-client/src/test/java/org/apache/hudi/client/TestFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/TestFlinkWriteClient.java new file mode 100644 index 0000000000000..0752e9e785c73 --- /dev/null +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/client/TestFlinkWriteClient.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client; + +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.HoodieFlinkClientTestHarness; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +public class TestFlinkWriteClient extends HoodieFlinkClientTestHarness { + + @BeforeEach + private void setup() throws IOException { + initPath(); + initFileSystem(); + initMetaClient(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testWriteClientAndTableServiceClientWithTimelineServer( + boolean enableEmbeddedTimelineServer) throws IOException { + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withPath(metaClient.getBasePathV2().toString()) + .withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineServer) + .build(); + + HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient(context, writeConfig); + // Only one timeline server should be instantiated, and the same timeline server + // should be used by both the write client and the table service client. + assertEquals( + writeClient.getTimelineServer(), + writeClient.getTableServiceClient().getTimelineServer()); + if (!enableEmbeddedTimelineServer) { + assertFalse(writeClient.getTimelineServer().isPresent()); + } + + writeClient.close(); + } +} diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java index 6dd5a1c27e212..d4b4007bedb19 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/index/bloom/TestFlinkHoodieBloomIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.data.HoodieListPairData; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -186,11 +187,14 @@ public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolea partitionRecordKeyMap.put(t.getLeft(), recordKeyList); }); - List> comparisonKeyList = index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieListPairData.lazy(partitionRecordKeyMap)).collectAsList(); + List> comparisonKeyList = + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieListPairData.lazy(partitionRecordKeyMap)).collectAsList(); assertEquals(10, comparisonKeyList.size()); java.util.Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(java.util.stream.Collectors.groupingBy(t -> t.getRight().getRecordKey(), java.util.stream.Collectors.mapping(t -> t.getLeft(), java.util.stream.Collectors.toList()))); + .collect( + java.util.stream.Collectors.groupingBy(t -> t.getRight(), + java.util.stream.Collectors.mapping(t -> t.getLeft().getFileId(), java.util.stream.Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(new java.util.HashSet<>(asList("f1", "f3", "f4")), new java.util.HashSet<>(recordKeyToFileComps.get("002"))); diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java index a1a07a65f9931..9e07edbd4ca05 100644 --- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java +++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/io/storage/row/parquet/TestParquetSchemaConverter.java @@ -56,7 +56,7 @@ void testConvertComplexTypes() { + " }\n" + " optional group f_map (MAP) {\n" + " repeated group key_value {\n" - + " optional int32 key;\n" + + " required int32 key;\n" + " optional binary value (STRING);\n" + " }\n" + " }\n" @@ -83,7 +83,7 @@ void testConvertTimestampTypes() { assertThat(messageType.getColumns().size(), is(3)); final String expected = "message converted {\n" + " optional int64 ts_3 (TIMESTAMP(MILLIS,true));\n" - + " optional int96 ts_6;\n" + + " optional int64 ts_6 (TIMESTAMP(MICROS,true));\n" + " optional int96 ts_9;\n" + "}\n"; assertThat(messageType.toString(), is(expected)); diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml index 00fc0c3513ca2..0b267bee3f032 100644 --- a/hudi-client/hudi-java-client/pom.xml +++ b/hudi-client/hudi-java-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-java-client - 0.13.0-SNAPSHOT + 0.13.1 hudi-java-client jar diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java index 2d823aa7f5779..bcbd7dac918f8 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java @@ -18,6 +18,7 @@ package org.apache.hudi.client; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.util.Option; @@ -34,8 +35,10 @@ public class HoodieJavaTableServiceClient extends BaseHoodieTableServiceClient> { - protected HoodieJavaTableServiceClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig); + protected HoodieJavaTableServiceClient(HoodieEngineContext context, + HoodieWriteConfig clientConfig, + Option timelineService) { + super(context, clientConfig, timelineService); } @Override 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 af35078b9a98d..997dd5d84e361 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 @@ -53,7 +53,7 @@ public class HoodieJavaWriteClient extends public HoodieJavaWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) { super(context, writeConfig, JavaUpgradeDowngradeHelper.getInstance()); - this.tableServiceClient = new HoodieJavaTableServiceClient(context, writeConfig); + this.tableServiceClient = new HoodieJavaTableServiceClient(context, writeConfig, getTimelineServer()); } public HoodieJavaWriteClient(HoodieEngineContext context, @@ -61,7 +61,7 @@ public HoodieJavaWriteClient(HoodieEngineContext context, boolean rollbackPending, Option timelineService) { super(context, writeConfig, timelineService, JavaUpgradeDowngradeHelper.getInstance()); - this.tableServiceClient = new HoodieJavaTableServiceClient(context, writeConfig); + this.tableServiceClient = new HoodieJavaTableServiceClient(context, writeConfig, getTimelineServer()); } @Override diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java index 5113b3406801e..d2e813a506bc8 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/JavaLazyInsertIterable.java @@ -64,7 +64,7 @@ protected List computeNext() { try { final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema()); bufferedIteratorExecutor = - ExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getCloningTransformer(schema)); + ExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), getTransformer(schema, hoodieConfig)); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); return result; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java index 540cac2ab777a..642cad4e21e71 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/execution/bulkinsert/JavaCustomColumnsSortPartitioner.java @@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.FlatLists; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.avro.Schema; @@ -51,9 +52,13 @@ public JavaCustomColumnsSortPartitioner(String[] columnNames, Schema schema, boo public List> repartitionRecords( List> records, int outputPartitions) { return records.stream().sorted((o1, o2) -> { - Object values1 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)o1, sortColumnNames, schema, consistentLogicalTimestampEnabled); - Object values2 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)o2, sortColumnNames, schema, consistentLogicalTimestampEnabled); - return values1.toString().compareTo(values2.toString()); + FlatLists.ComparableList values1 = FlatLists.ofComparableArray( + HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord) o1, sortColumnNames, schema, consistentLogicalTimestampEnabled) + ); + FlatLists.ComparableList values2 = FlatLists.ofComparableArray( + HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord) o2, sortColumnNames, schema, consistentLogicalTimestampEnabled) + ); + return values1.compareTo(values2); }).collect(Collectors.toList()); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java index f3046bf22e20c..04befe4ea1627 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java @@ -79,7 +79,7 @@ protected List> doDeduplicateRecords( // we cannot allow the user to change the key or partitionPath, since that will affect // everything // so pick it from one of the records. - return reducedRecord.newInstance(rec1.getKey()); + return reducedRecord.newInstance(rec1.getKey(), rec1.getOperation()); }).orElse(null)).filter(Objects::nonNull).collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java index ae73b0a65d795..02c407ba02db3 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestHoodieJavaWriteClientInsert.java @@ -19,6 +19,7 @@ package org.apache.hudi.client; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.FSUtils; @@ -31,6 +32,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; import org.apache.hudi.common.util.BaseFileUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; @@ -44,8 +47,10 @@ import org.apache.hadoop.mapred.JobConf; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; +import java.io.IOException; import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; @@ -53,7 +58,9 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieJavaWriteClientInsert extends HoodieJavaClientTestHarness { @@ -96,6 +103,45 @@ private void setupIncremental(JobConf jobConf, String startCommit, int numberOfC jobConf.setInt(maxCommitPulls, numberOfCommitsToPull); } + @ParameterizedTest + @CsvSource({"true,true", "true,false", "false,true", "false,false"}) + public void testWriteClientAndTableServiceClientWithTimelineServer( + boolean enableEmbeddedTimelineServer, boolean passInTimelineServer) throws IOException { + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(INMEMORY).build()) + .withPath(metaClient.getBasePathV2().toString()) + .withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineServer) + .build(); + + HoodieJavaWriteClient writeClient; + if (passInTimelineServer) { + EmbeddedTimelineService timelineService = + new EmbeddedTimelineService(context, null, writeConfig); + timelineService.startServer(); + writeConfig.setViewStorageConfig(timelineService.getRemoteFileSystemViewConfig()); + writeClient = new HoodieJavaWriteClient(context, writeConfig, true, Option.of(timelineService)); + // Both the write client and the table service client should use the same passed-in + // timeline server instance. + assertEquals(timelineService, writeClient.getTimelineServer().get()); + assertEquals(timelineService, writeClient.getTableServiceClient().getTimelineServer().get()); + // Write config should not be changed + assertEquals(writeConfig, writeClient.getConfig()); + timelineService.stop(); + } else { + writeClient = new HoodieJavaWriteClient(context, writeConfig); + // Only one timeline server should be instantiated, and the same timeline server + // should be used by both the write client and the table service client. + assertEquals( + writeClient.getTimelineServer(), + writeClient.getTableServiceClient().getTimelineServer()); + if (!enableEmbeddedTimelineServer) { + assertFalse(writeClient.getTimelineServer().isPresent()); + } + } + writeClient.close(); + } + @Test public void testInsert() throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder(basePath).withMergeAllowDuplicateOnInserts(true).build(); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java index e4a30ba84120a..b85565e5027c6 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestJavaBulkInsertInternalPartitioner.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.FlatLists; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.testutils.HoodieJavaClientTestHarness; @@ -70,8 +71,9 @@ public void testCustomColumnSortPartitioner(String sortColumnString) throws Exce } private Comparator getCustomColumnComparator(Schema schema, String[] sortColumns) { - return Comparator.comparing( - record -> HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)record, sortColumns, schema, false).toString()); + return Comparator.comparing(record -> + FlatLists.ofComparableArray( + HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)record, sortColumns, schema, false))); } private void verifyRecordAscendingOrder(List records, diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml index 5075bc687367c..19a8ae91db08a 100644 --- a/hudi-client/hudi-spark-client/pom.xml +++ b/hudi-client/hudi-spark-client/pom.xml @@ -19,12 +19,12 @@ hudi-client org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-spark-client - 0.13.0-SNAPSHOT + 0.13.1 hudi-spark-client jar diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java index 1fcd6acb12de4..de203a0aa5b62 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java @@ -20,6 +20,7 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -58,8 +59,10 @@ public class SparkRDDTableServiceClient extends BaseHoodieTableServiceClient< private static final Logger LOG = LoggerFactory.getLogger(SparkRDDTableServiceClient.class); - protected SparkRDDTableServiceClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) { - super(context, clientConfig); + protected SparkRDDTableServiceClient(HoodieEngineContext context, + HoodieWriteConfig clientConfig, + Option timelineService) { + super(context, clientConfig, timelineService); } @Override 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 86f559260a791..8b260bb377c43 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 @@ -82,7 +82,7 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, Option timelineService) { super(context, writeConfig, timelineService, SparkUpgradeDowngradeHelper.getInstance()); - this.tableServiceClient = new SparkRDDTableServiceClient<>(context, writeConfig); + this.tableServiceClient = new SparkRDDTableServiceClient<>(context, writeConfig, getTimelineServer()); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index b161182b83a36..bc0a1663c4bc4 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -60,7 +60,7 @@ protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List> readRecordsForGroupWithLogs(JavaSparkContext String instantTime) { HoodieWriteConfig config = getWriteConfig(); HoodieTable table = getHoodieTable(); + // NOTE: It's crucial to make sure that we don't capture whole "this" object into the + // closure, as this might lead to issues attempting to serialize its nested fields + SerializableConfiguration hadoopConf = new SerializableConfiguration(table.getHadoopConf()); + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + String bootstrapBasePath = tableConfig.getBootstrapBasePath().orElse(null); + Option partitionFields = tableConfig.getPartitionFields(); + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { List>> recordIterators = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { @@ -293,7 +305,7 @@ private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext .withBufferSize(config.getMaxDFSStreamBufferSize()) .withSpillableMapBasePath(config.getSpillableMapBasePath()) .withPartition(clusteringOp.getPartitionPath()) - .withUseScanV2(config.useScanV2ForLogRecordReader()) + .withOptimizedLogBlocksScan(config.enableOptimizedLogBlocksScan()) .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType()) .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled()) .withRecordMerger(config.getRecordMerger()) @@ -301,8 +313,7 @@ private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext Option baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath()) ? Option.empty() - : Option.of(HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))); - HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + : Option.of(getBaseOrBootstrapFileReader(hadoopConf, bootstrapBasePath, partitionFields, clusteringOp)); recordIterators.add(getFileSliceReader(baseFileReader, scanner, readerSchema, tableConfig.getProps(), tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(), @@ -327,13 +338,18 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex // NOTE: It's crucial to make sure that we don't capture whole "this" object into the // closure, as this might lead to issues attempting to serialize its nested fields + HoodieTableConfig tableConfig = getHoodieTable().getMetaClient().getTableConfig(); + String bootstrapBasePath = tableConfig.getBootstrapBasePath().orElse(null); + Option partitionFields = tableConfig.getPartitionFields(); + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()) .mapPartitions(clusteringOpsPartition -> { List>> iteratorsForPartition = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { try { Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema())); - HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); + HoodieFileReader baseFileReader = getBaseOrBootstrapFileReader(hadoopConf, bootstrapBasePath, partitionFields, clusteringOp); + Option keyGeneratorOp = writeConfig.populateMetaFields() ? Option.empty() : Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps())); // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific @@ -353,6 +369,35 @@ private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContex })); } + private HoodieFileReader getBaseOrBootstrapFileReader(SerializableConfiguration hadoopConf, String bootstrapBasePath, Option partitionFields, ClusteringOperation clusteringOp) + throws IOException { + HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath())); + // handle bootstrap path + if (StringUtils.nonEmpty(clusteringOp.getBootstrapFilePath()) && StringUtils.nonEmpty(bootstrapBasePath)) { + String bootstrapFilePath = clusteringOp.getBootstrapFilePath(); + Object[] partitionValues = new Object[0]; + if (partitionFields.isPresent()) { + int startOfPartitionPath = bootstrapFilePath.indexOf(bootstrapBasePath) + bootstrapBasePath.length() + 1; + String partitionFilePath = bootstrapFilePath.substring(startOfPartitionPath, bootstrapFilePath.lastIndexOf("/")); + CachingPath bootstrapCachingPath = new CachingPath(bootstrapBasePath); + SparkParsePartitionUtil sparkParsePartitionUtil = SparkAdapterSupport$.MODULE$.sparkAdapter().getSparkParsePartitionUtil(); + partitionValues = HoodieSparkUtils.parsePartitionColumnValues( + partitionFields.get(), + partitionFilePath, + bootstrapCachingPath, + AvroConversionUtils.convertAvroSchemaToStructType(baseFileReader.getSchema()), + hadoopConf.get().get("timeZone", SQLConf.get().sessionLocalTimeZone()), + sparkParsePartitionUtil, + hadoopConf.get().getBoolean("spark.sql.sources.validatePartitionColumns", true)); + } + baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).newBootstrapFileReader( + baseFileReader, + HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(bootstrapFilePath)), partitionFields, + partitionValues); + } + return baseFileReader; + } + /** * Get dataset of all records for the group. This includes all records from file slice (Apply updates from log files, if any). */ @@ -368,9 +413,8 @@ private Dataset readRecordsForGroupAsRow(JavaSparkContext jsc, .stream() .map(op -> { ArrayList readPaths = new ArrayList<>(); - if (op.getBootstrapFilePath() != null) { - readPaths.add(op.getBootstrapFilePath()); - } + // NOTE: for bootstrap tables, only need to handle data file path (which is the skeleton file) because + // HoodieBootstrapRelation takes care of stitching if there is bootstrap path for the skeleton file. if (op.getDataFilePath() != null) { readPaths.add(op.getDataFilePath()); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java index 198b097134bae..2b14bb3a0665b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkInternalSchemaConverter.java @@ -287,7 +287,7 @@ private static DataType constructSparkSchemaFromType(Type type) { /** * Convert Int/long type to other Type. - * Now only support int/long -> long/float/double/string + * Now only support int/long -> long/float/double/string/Decimal * TODO: support more types */ private static boolean convertIntLongType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) { @@ -321,7 +321,7 @@ private static boolean convertIntLongType(WritableColumnVector oldV, WritableCol /** * Convert float type to other Type. - * Now only support float -> double/String + * Now only support float -> double/String/Decimal * TODO: support more types */ private static boolean convertFloatType(WritableColumnVector oldV, WritableColumnVector newV, DataType newType, int len) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java index b119e75e2177c..7756d2502e897 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/common/model/HoodieSparkRecord.java @@ -30,9 +30,9 @@ 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.exception.HoodieException; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.SparkKeyGeneratorInterface; -import org.apache.hudi.util.HoodieSparkRecordUtils; import org.apache.spark.sql.HoodieInternalRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils; import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; @@ -44,14 +44,13 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +import scala.Function1; import java.io.IOException; import java.util.Map; import java.util.Properties; import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; -import static org.apache.hudi.util.HoodieSparkRecordUtils.getNullableValAsString; -import static org.apache.hudi.util.HoodieSparkRecordUtils.getValue; import static org.apache.spark.sql.types.DataTypes.BooleanType; import static org.apache.spark.sql.types.DataTypes.StringType; @@ -150,8 +149,9 @@ public String getRecordKey(Schema recordSchema, Option keyGene return getRecordKey(); } StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); - return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()) - .getRecordKey(data, structType).toString() : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal()); + return keyGeneratorOpt.isPresent() + ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType).toString() + : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal()); } @Override @@ -173,7 +173,11 @@ public HoodieRecordType getRecordType() { @Override public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); - return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled); + Object[] objects = new Object[columns.length]; + for (int i = 0; i < objects.length; i++) { + objects[i] = getValue(structType, columns[i], data); + } + return objects; } @Override @@ -186,50 +190,27 @@ public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { } @Override - public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + public HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema); - // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter - InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecord(this.data, structType, targetStructType); - UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(targetStructType, targetStructType).apply(rewriteRecord); - - boolean containMetaFields = hasMetaFields(targetStructType); - UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, targetStructType); - HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); + HoodieInternalRow updatableRow = wrapIntoUpdatableOverlay(this.data, structType); + updateMetadataValuesInternal(updatableRow, metadataValues); - return new HoodieSparkRecord(getKey(), internalRow, targetStructType, getOperation(), this.currentLocation, this.newLocation, false); + return new HoodieSparkRecord(getKey(), updatableRow, targetStructType, getOperation(), this.currentLocation, this.newLocation, false); } @Override - public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema); - // TODO HUDI-5281 Rewrite HoodieSparkRecord with UnsafeRowWriter - InternalRow rewriteRecord = HoodieInternalRowUtils.rewriteRecordWithNewSchema(this.data, structType, newStructType, renameCols); - UnsafeRow unsafeRow = HoodieInternalRowUtils.getCachedUnsafeProjection(newStructType, newStructType).apply(rewriteRecord); + Function1 unsafeRowWriter = + HoodieInternalRowUtils.getCachedUnsafeRowWriter(structType, newStructType, renameCols); - boolean containMetaFields = hasMetaFields(newStructType); - UTF8String[] metaFields = tryExtractMetaFields(unsafeRow, newStructType); - HoodieInternalRow internalRow = new HoodieInternalRow(metaFields, unsafeRow, containMetaFields); + UnsafeRow unsafeRow = unsafeRowWriter.apply(this.data); - return new HoodieSparkRecord(getKey(), internalRow, newStructType, getOperation(), this.currentLocation, this.newLocation, false); - } - - @Override - public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { - StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); - HoodieInternalRow updatableRow = wrapIntoUpdatableOverlay(data, structType); - - metadataValues.getKv().forEach((key, value) -> { - int pos = structType.fieldIndex(key); - if (value != null) { - updatableRow.update(pos, CatalystTypeConverters.convertToCatalyst(value)); - } - }); - - return new HoodieSparkRecord(getKey(), updatableRow, structType, getOperation(), this.currentLocation, this.newLocation, copy); + return new HoodieSparkRecord(getKey(), unsafeRow, newStructType, getOperation(), this.currentLocation, this.newLocation, false); } @Override @@ -317,12 +298,13 @@ public HoodieSparkRecord copy() { public Comparable getOrderingValue(Schema recordSchema, Properties props) { StructType structType = HoodieInternalRowUtils.getCachedSchema(recordSchema); String orderingField = ConfigUtils.getOrderingField(props); - if (!HoodieInternalRowUtils.existField(structType, orderingField)) { - return 0; + scala.Option cachedNestedFieldPath = + HoodieInternalRowUtils.getCachedPosList(structType, orderingField); + if (cachedNestedFieldPath.isDefined()) { + NestedFieldPath nestedFieldPath = cachedNestedFieldPath.get(); + return (Comparable) HoodieUnsafeRowUtils.getNestedInternalRowValue(data, nestedFieldPath); } else { - NestedFieldPath nestedFieldPath = HoodieInternalRowUtils.getCachedPosList(structType, orderingField); - Comparable value = (Comparable) HoodieUnsafeRowUtils.getNestedInternalRowValue(data, nestedFieldPath); - return value; + return 0; } } @@ -368,21 +350,28 @@ private static HoodieInternalRow wrapIntoUpdatableOverlay(InternalRow data, Stru } boolean containsMetaFields = hasMetaFields(structType); - UTF8String[] metaFields = tryExtractMetaFields(data, structType); + UTF8String[] metaFields = extractMetaFields(data, structType); return new HoodieInternalRow(metaFields, data, containsMetaFields); } - private static UTF8String[] tryExtractMetaFields(InternalRow row, StructType structType) { + private static UTF8String[] extractMetaFields(InternalRow row, StructType structType) { boolean containsMetaFields = hasMetaFields(structType); - if (containsMetaFields && structType.size() == 1) { - // Support bootstrap with RECORD_KEY_SCHEMA - return new UTF8String[] {row.getUTF8String(0)}; - } else if (containsMetaFields) { + if (containsMetaFields) { return HoodieRecord.HOODIE_META_COLUMNS.stream() .map(col -> row.getUTF8String(HOODIE_META_COLUMNS_NAME_TO_POS.get(col))) .toArray(UTF8String[]::new); - } else { - return new UTF8String[HoodieRecord.HOODIE_META_COLUMNS.size()]; + } + + return new UTF8String[HoodieRecord.HOODIE_META_COLUMNS.size()]; + } + + private static void updateMetadataValuesInternal(HoodieInternalRow updatableRow, MetadataValues metadataValues) { + String[] values = metadataValues.getValues(); + for (int pos = 0; pos < values.length; ++pos) { + String value = values[pos]; + if (value != null) { + updatableRow.update(pos, CatalystTypeConverters.convertToCatalyst(value)); + } } } @@ -416,7 +405,8 @@ private static HoodieRecord convertToHoodieSparkRecord(StructType s getValue(structType, recordKeyPartitionPathFieldPair.getRight(), record.data).toString()); HoodieOperation operation = withOperationField - ? HoodieOperation.fromName(getNullableValAsString(structType, record.data, HoodieRecord.OPERATION_METADATA_FIELD)) : null; + ? HoodieOperation.fromName(record.data.getString(structType.fieldIndex(HoodieRecord.OPERATION_METADATA_FIELD))) + : null; return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), record.data, structType, operation, record.copy); } @@ -434,4 +424,14 @@ private static void validateRow(InternalRow data, StructType schema) { ValidationUtils.checkState(isValid); } + + private static Object getValue(StructType structType, String fieldName, InternalRow row) { + scala.Option cachedNestedFieldPath = + HoodieInternalRowUtils.getCachedPosList(structType, fieldName); + if (cachedNestedFieldPath.isDefined()) { + return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, cachedNestedFieldPath.get()); + } else { + throw new HoodieException(String.format("Field at %s is not present in %s", fieldName, structType)); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java index 9ec3c4cf71592..9019fb43ff058 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java @@ -121,6 +121,11 @@ public HoodieData map(SerializableFunction, O> func) { tuple -> func.apply(new ImmutablePair<>(tuple._1, tuple._2)))); } + @Override + public HoodiePairData mapValues(SerializableFunction func) { + return HoodieJavaPairRDD.of(pairRDDData.mapValues(func::apply)); + } + @Override public HoodiePairData mapToPair(SerializablePairFunction, L, W> mapToPairFunc) { return HoodieJavaPairRDD.of(pairRDDData.mapToPair(pair -> { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java index ed9613bc15fe6..6ed3a854962b8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java @@ -24,17 +24,16 @@ import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; +import org.apache.hudi.common.util.collection.MappingIterator; import org.apache.hudi.common.util.collection.Pair; - import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.storage.StorageLevel; +import scala.Tuple2; import java.util.Iterator; import java.util.List; -import scala.Tuple2; - /** * Holds a {@link JavaRDD} of objects. * @@ -119,9 +118,18 @@ public HoodieData mapPartitions(SerializableFunction, Iterato @Override public HoodieData flatMap(SerializableFunction> func) { + // NOTE: Unrolling this lambda into a method reference results in [[ClassCastException]] + // due to weird interop b/w Scala and Java return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e))); } + @Override + public HoodiePairData flatMapToPair(SerializableFunction>> func) { + return HoodieJavaPairRDD.of( + rddData.flatMapToPair(e -> + new MappingIterator<>(func.apply(e), p -> new Tuple2<>(p.getKey(), p.getValue())))); + } + @Override public HoodiePairData mapToPair(SerializablePairFunction func) { return HoodieJavaPairRDD.of(rddData.mapToPair(input -> { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java index 7cb2b27e17115..3b42d40a1a22a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java @@ -38,19 +38,7 @@ public class SparkLazyInsertIterable extends HoodieLazyInsertIterable { - private boolean useWriterSchema; - - public SparkLazyInsertIterable(Iterator> recordItr, - boolean areRecordsSorted, - HoodieWriteConfig config, - String instantTime, - HoodieTable hoodieTable, - String idPrefix, - TaskContextSupplier taskContextSupplier, - boolean useWriterSchema) { - super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier); - this.useWriterSchema = useWriterSchema; - } + private final boolean useWriterSchema; public SparkLazyInsertIterable(Iterator> recordItr, boolean areRecordsSorted, @@ -87,7 +75,7 @@ protected List computeNext() { } bufferedIteratorExecutor = ExecutorFactory.create(hoodieConfig, inputItr, getInsertHandler(), - getCloningTransformer(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); + getTransformer(schema, hoodieConfig), hoodieTable.getPreExecuteRunnable()); final List result = bufferedIteratorExecutor.execute(); checkState(result != null && !result.isEmpty()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java index e723d724b6405..4b988200de4af 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java @@ -20,7 +20,7 @@ import org.apache.hudi.common.config.SerializableSchema; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.FlatLists; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.BulkInsertPartitioner; @@ -30,7 +30,7 @@ import java.util.Arrays; /** - * A partitioner that does sorting based on specified column values for each RDD partition. + * A partitioner that does sort based on specified column values for each RDD partition. * * @param HoodieRecordPayload type */ @@ -61,13 +61,8 @@ public JavaRDD> repartitionRecords(JavaRDD> reco final boolean consistentLogicalTimestampEnabled = this.consistentLogicalTimestampEnabled; return records.sortBy( record -> { - Object recordValue = record.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled); - // null values are replaced with empty string for null_first order - if (recordValue == null) { - return StringUtils.EMPTY_STRING; - } else { - return StringUtils.objToString(recordValue); - } + Object[] columnValues = record.getColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled); + return FlatLists.ofComparableArray(columnValues); }, true, outputSparkPartitions); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java index 36710dc02bb9b..48099220dbf92 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java @@ -18,6 +18,7 @@ package org.apache.hudi.index.bloom; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.collection.Pair; @@ -63,7 +64,7 @@ public class BucketizedBloomCheckPartitioner extends Partitioner { /** * Stores the final mapping of a file group to a list of partitions for its keys. */ - private Map> fileGroupToPartitions; + private Map> fileGroupToPartitions; /** * Create a partitioner that computes a plan based on provided workload characteristics. @@ -72,11 +73,11 @@ public class BucketizedBloomCheckPartitioner extends Partitioner { * @param fileGroupToComparisons number of expected comparisons per file group * @param keysPerBucket maximum number of keys to pack in a single bucket */ - public BucketizedBloomCheckPartitioner(int targetPartitions, Map fileGroupToComparisons, + public BucketizedBloomCheckPartitioner(int targetPartitions, Map fileGroupToComparisons, int keysPerBucket) { this.fileGroupToPartitions = new HashMap<>(); - Map bucketsPerFileGroup = new HashMap<>(); + Map bucketsPerFileGroup = new HashMap<>(); // Compute the buckets needed per file group, using simple uniform distribution fileGroupToComparisons.forEach((f, c) -> bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket))); int totalBuckets = bucketsPerFileGroup.values().stream().mapToInt(i -> i).sum(); @@ -90,9 +91,9 @@ public BucketizedBloomCheckPartitioner(int targetPartitions, Map f int minBucketsPerPartition = Math.max((int) Math.floor((1.0 * totalBuckets) / partitions), 1); LOG.info(String.format("TotalBuckets %d, min_buckets/partition %d", totalBuckets, minBucketsPerPartition)); int[] bucketsFilled = new int[partitions]; - Map bucketsFilledPerFileGroup = new HashMap<>(); + Map bucketsFilledPerFileGroup = new HashMap<>(); int partitionIndex = 0; - for (Map.Entry e : bucketsPerFileGroup.entrySet()) { + for (Map.Entry e : bucketsPerFileGroup.entrySet()) { for (int b = 0; b < Math.max(1, e.getValue() - 1); b++) { // keep filled counts upto date bucketsFilled[partitionIndex]++; @@ -115,7 +116,7 @@ public BucketizedBloomCheckPartitioner(int targetPartitions, Map f // PHASE 2 : for remaining unassigned buckets, round robin over partitions once. Since we withheld 1 bucket from // each file group uniformly, this remaining is also an uniform mix across file groups. We just round robin to // optimize for goal 2. - for (Map.Entry e : bucketsPerFileGroup.entrySet()) { + for (Map.Entry e : bucketsPerFileGroup.entrySet()) { int remaining = e.getValue() - bucketsFilledPerFileGroup.get(e.getKey()).intValue(); for (int r = 0; r < remaining; r++) { // mark this partition against the file group @@ -142,7 +143,8 @@ public int numPartitions() { @Override public int getPartition(Object key) { - final Pair parts = (Pair) key; + final Pair parts = (Pair) key; + // TODO replace w/ more performant hash final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", parts.getRight()); final List candidatePartitions = fileGroupToPartitions.get(parts.getLeft()); final int idx = (int) Math.floorMod((int) hashOfKey, candidatePartitions.size()); @@ -150,7 +152,7 @@ public int getPartition(Object key) { return candidatePartitions.get(idx); } - Map> getFileGroupToPartitions() { + Map> getFileGroupToPartitions() { return fileGroupToPartitions; } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomFilterProbingResult.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomFilterProbingResult.java new file mode 100644 index 0000000000000..c124f8b27b80f --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomFilterProbingResult.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.index.bloom; + +import java.util.List; + +class HoodieBloomFilterProbingResult { + + private final List candidateKeys; + + HoodieBloomFilterProbingResult(List candidateKeys) { + this.candidateKeys = candidateKeys; + } + + public List getCandidateKeys() { + return candidateKeys; + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java deleted file mode 100644 index e19a429ea7234..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.bloom; - -import org.apache.hudi.client.utils.LazyIterableIterator; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.io.HoodieKeyLookupHandle; -import org.apache.hudi.io.HoodieKeyLookupResult; -import org.apache.hudi.table.HoodieTable; - -import org.apache.spark.api.java.function.Function2; - -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import scala.Tuple2; - -/** - * Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files. - */ -public class HoodieBloomIndexCheckFunction - implements Function2>, Iterator>> { - - private final HoodieTable hoodieTable; - - private final HoodieWriteConfig config; - - public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) { - this.hoodieTable = hoodieTable; - this.config = config; - } - - @Override - public Iterator> call(Integer partition, - Iterator> filePartitionRecordKeyTripletItr) { - return new LazyKeyCheckIterator(filePartitionRecordKeyTripletItr); - } - - class LazyKeyCheckIterator extends LazyIterableIterator, List> { - - private HoodieKeyLookupHandle keyLookupHandle; - - LazyKeyCheckIterator(Iterator> filePartitionRecordKeyTripletItr) { - super(filePartitionRecordKeyTripletItr); - } - - @Override - protected void start() { - } - - @Override - protected List computeNext() { - - List ret = new ArrayList<>(); - try { - // process one file in each go. - while (inputItr.hasNext()) { - Tuple2 currentTuple = inputItr.next(); - String fileId = currentTuple._1; - String partitionPath = currentTuple._2.getPartitionPath(); - String recordKey = currentTuple._2.getRecordKey(); - Pair partitionPathFilePair = Pair.of(partitionPath, fileId); - - // lazily init state - if (keyLookupHandle == null) { - keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair); - } - - // if continue on current file - if (keyLookupHandle.getPartitionPathFileIDPair().equals(partitionPathFilePair)) { - keyLookupHandle.addKey(recordKey); - } else { - // do the actual checking of file & break out - ret.add(keyLookupHandle.getLookupResult()); - keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair); - keyLookupHandle.addKey(recordKey); - break; - } - } - - // handle case, where we ran out of input, close pending work, update return val - if (!inputItr.hasNext()) { - ret.add(keyLookupHandle.getLookupResult()); - } - } catch (Throwable e) { - if (e instanceof HoodieException) { - throw e; - } - throw new HoodieIndexException("Error checking bloom filter index. ", e); - } - - return ret; - } - - @Override - protected void end() { - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieFileProbingFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieFileProbingFunction.java new file mode 100644 index 0000000000000..0809042c9fbbf --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieFileProbingFunction.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bloom; + +import org.apache.hadoop.fs.Path; +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.config.SerializableConfiguration; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +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.HoodieIndexException; +import org.apache.hudi.index.HoodieIndexUtils; +import org.apache.hudi.io.HoodieKeyLookupResult; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import scala.Tuple2; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Implementation of the function probing filtered in candidate keys provided in + * {@link HoodieBloomFilterProbingResult} w/in corresponding files identified by {@link HoodieFileGroupId} + * to validate whether the record w/ the provided key is indeed persisted in it + */ +public class HoodieFileProbingFunction implements + FlatMapFunction>, List> { + + private static final Logger LOG = LogManager.getLogger(HoodieFileProbingFunction.class); + + // Assuming each file bloom filter takes up 512K, sizing the max file count + // per batch so that the total fetched bloom filters would not cross 128 MB. + private static final long BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH = 256; + + private final Broadcast baseFileOnlyViewBroadcast; + private final SerializableConfiguration hadoopConf; + + public HoodieFileProbingFunction(Broadcast baseFileOnlyViewBroadcast, + SerializableConfiguration hadoopConf) { + this.baseFileOnlyViewBroadcast = baseFileOnlyViewBroadcast; + this.hadoopConf = hadoopConf; + } + + @Override + public Iterator> call(Iterator> tuple2Iterator) throws Exception { + return new BloomIndexLazyKeyCheckIterator(tuple2Iterator); + } + + private class BloomIndexLazyKeyCheckIterator + extends LazyIterableIterator, List> { + + public BloomIndexLazyKeyCheckIterator(Iterator> tuple2Iterator) { + super(tuple2Iterator); + } + + @Override + protected List computeNext() { + // Partition path and file name pair to list of keys + final Map, HoodieBloomFilterProbingResult> fileToLookupResults = new HashMap<>(); + final Map fileIDBaseFileMap = new HashMap<>(); + + while (inputItr.hasNext()) { + Tuple2 entry = inputItr.next(); + final String partitionPath = entry._1.getPartitionPath(); + final String fileId = entry._1.getFileId(); + + if (!fileIDBaseFileMap.containsKey(fileId)) { + Option baseFile = + baseFileOnlyViewBroadcast.getValue().getLatestBaseFile(partitionPath, fileId); + if (!baseFile.isPresent()) { + throw new HoodieIndexException("Failed to find the base file for partition: " + partitionPath + + ", fileId: " + fileId); + } + + fileIDBaseFileMap.put(fileId, baseFile.get()); + } + + fileToLookupResults.putIfAbsent(Pair.of(partitionPath, fileIDBaseFileMap.get(fileId).getFileName()), entry._2); + + if (fileToLookupResults.size() > BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH) { + break; + } + } + + if (fileToLookupResults.isEmpty()) { + return Collections.emptyList(); + } + + return fileToLookupResults.entrySet().stream() + .map(entry -> { + Pair partitionPathFileNamePair = entry.getKey(); + HoodieBloomFilterProbingResult bloomFilterKeyLookupResult = entry.getValue(); + + final String partitionPath = partitionPathFileNamePair.getLeft(); + final String fileName = partitionPathFileNamePair.getRight(); + final String fileId = FSUtils.getFileId(fileName); + ValidationUtils.checkState(!fileId.isEmpty()); + + List candidateRecordKeys = bloomFilterKeyLookupResult.getCandidateKeys(); + + // TODO add assertion that file is checked only once + + final HoodieBaseFile dataFile = fileIDBaseFileMap.get(fileId); + List matchingKeys = HoodieIndexUtils.filterKeysFromFile(new Path(dataFile.getPath()), + candidateRecordKeys, hadoopConf.get()); + + LOG.debug( + String.format("Bloom filter candidates (%d) / false positives (%d), actual matches (%d)", + candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size())); + + return new HoodieKeyLookupResult(fileId, partitionPath, dataFile.getCommitTime(), matchingKeys); + }) + .collect(Collectors.toList()); + } + + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomFilterProbingFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomFilterProbingFunction.java new file mode 100644 index 0000000000000..406be81650057 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomFilterProbingFunction.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.index.bloom; + +import org.apache.hudi.client.utils.LazyIterableIterator; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.FlatteningIterator; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.table.HoodieTable; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import scala.Tuple2; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * Implementation of the function that probing Bloom Filters of individual files verifying + * whether particular record key could be stored in the latest file-slice of the file-group + * identified by the {@link HoodieFileGroupId} + */ +public class HoodieMetadataBloomFilterProbingFunction implements + PairFlatMapFunction>, HoodieFileGroupId, HoodieBloomFilterProbingResult> { + + private static final Logger LOG = LogManager.getLogger(HoodieMetadataBloomFilterProbingFunction.class); + + // Assuming each file bloom filter takes up 512K, sizing the max file count + // per batch so that the total fetched bloom filters would not cross 128 MB. + private static final long BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH = 256; + private final HoodieTable hoodieTable; + + private final Broadcast baseFileOnlyViewBroadcast; + + /** + * NOTE: It's critical for this ctor to accept {@link HoodieTable} to make sure that it uses + * broadcast-ed instance of {@link HoodieBackedTableMetadata} internally, instead of + * one being serialized and deserialized for _every_ task individually + * + * NOTE: We pass in broadcasted {@link HoodieTableFileSystemView} to make sure it's materialized + * on executor once + */ + public HoodieMetadataBloomFilterProbingFunction(Broadcast baseFileOnlyViewBroadcast, + HoodieTable hoodieTable) { + this.baseFileOnlyViewBroadcast = baseFileOnlyViewBroadcast; + this.hoodieTable = hoodieTable; + } + + @Override + public Iterator> call(Iterator> tuple2Iterator) throws Exception { + return new FlatteningIterator<>(new BloomIndexLazyKeyCheckIterator(tuple2Iterator)); + } + + private class BloomIndexLazyKeyCheckIterator + extends LazyIterableIterator, Iterator>> { + + public BloomIndexLazyKeyCheckIterator(Iterator> tuple2Iterator) { + super(tuple2Iterator); + } + + @Override + protected Iterator> computeNext() { + // Partition path and file name pair to list of keys + final Map, List> fileToKeysMap = new HashMap<>(); + final Map fileIDBaseFileMap = new HashMap<>(); + + while (inputItr.hasNext()) { + Tuple2 entry = inputItr.next(); + String partitionPath = entry._1.getPartitionPath(); + String fileId = entry._1.getFileId(); + + if (!fileIDBaseFileMap.containsKey(fileId)) { + Option baseFile = baseFileOnlyViewBroadcast.getValue().getLatestBaseFile(partitionPath, fileId); + if (!baseFile.isPresent()) { + throw new HoodieIndexException("Failed to find the base file for partition: " + partitionPath + + ", fileId: " + fileId); + } + fileIDBaseFileMap.put(fileId, baseFile.get()); + } + + fileToKeysMap.computeIfAbsent(Pair.of(partitionPath, fileIDBaseFileMap.get(fileId).getFileName()), + k -> new ArrayList<>()).add(new HoodieKey(entry._2, partitionPath)); + + if (fileToKeysMap.size() > BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH) { + break; + } + } + + if (fileToKeysMap.isEmpty()) { + return Collections.emptyIterator(); + } + + List> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet()); + Map, BloomFilter> fileToBloomFilterMap = + hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList); + + return fileToKeysMap.entrySet().stream() + .map(entry -> { + Pair partitionPathFileNamePair = entry.getKey(); + List hoodieKeyList = entry.getValue(); + + final String partitionPath = partitionPathFileNamePair.getLeft(); + final String fileName = partitionPathFileNamePair.getRight(); + final String fileId = FSUtils.getFileId(fileName); + ValidationUtils.checkState(!fileId.isEmpty()); + + if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) { + throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair); + } + final BloomFilter fileBloomFilter = fileToBloomFilterMap.get(partitionPathFileNamePair); + + List candidateRecordKeys = new ArrayList<>(); + hoodieKeyList.forEach(hoodieKey -> { + if (fileBloomFilter.mightContain(hoodieKey.getRecordKey())) { + candidateRecordKeys.add(hoodieKey.getRecordKey()); + } + }); + + LOG.debug(String.format("Total records (%d), bloom filter candidates (%d)", + hoodieKeyList.size(), candidateRecordKeys.size())); + + return Tuple2.apply(new HoodieFileGroupId(partitionPath, fileId), new HoodieBloomFilterProbingResult(candidateRecordKeys)); + }) + .iterator(); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java deleted file mode 100644 index 8a2958eab9da8..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieMetadataBloomIndexCheckFunction.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.index.bloom; - -import org.apache.hadoop.fs.Path; -import org.apache.hudi.client.utils.LazyIterableIterator; -import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieKey; -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.HoodieIndexException; -import org.apache.hudi.index.HoodieIndexUtils; -import org.apache.hudi.io.HoodieKeyLookupResult; -import org.apache.hudi.table.HoodieTable; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.spark.api.java.function.Function2; -import scala.Tuple2; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Spark Function2 implementation for checking bloom filters for the - * requested keys from the metadata table index. The bloom filter - * checking for keys and the actual file verification for the - * candidate keys is done in an iterative fashion. In each iteration, - * bloom filters are requested for a batch of partition files and the - * keys are checked against them. - */ -public class HoodieMetadataBloomIndexCheckFunction implements - Function2>, Iterator>> { - - private static final Logger LOG = LogManager.getLogger(HoodieMetadataBloomIndexCheckFunction.class); - - // Assuming each file bloom filter takes up 512K, sizing the max file count - // per batch so that the total fetched bloom filters would not cross 128 MB. - private static final long BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH = 256; - private final HoodieTable hoodieTable; - - public HoodieMetadataBloomIndexCheckFunction(HoodieTable hoodieTable) { - this.hoodieTable = hoodieTable; - } - - @Override - public Iterator> call(Integer integer, Iterator> tuple2Iterator) throws Exception { - return new BloomIndexLazyKeyCheckIterator(tuple2Iterator); - } - - private class BloomIndexLazyKeyCheckIterator extends LazyIterableIterator, List> { - public BloomIndexLazyKeyCheckIterator(Iterator> tuple2Iterator) { - super(tuple2Iterator); - } - - @Override - protected void start() { - } - - @Override - protected List computeNext() { - // Partition path and file name pair to list of keys - final Map, List> fileToKeysMap = new HashMap<>(); - final Map fileIDBaseFileMap = new HashMap<>(); - final List resultList = new ArrayList<>(); - - while (inputItr.hasNext()) { - Tuple2 entry = inputItr.next(); - final String partitionPath = entry._2.getPartitionPath(); - final String fileId = entry._1; - if (!fileIDBaseFileMap.containsKey(fileId)) { - Option baseFile = hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId); - if (!baseFile.isPresent()) { - throw new HoodieIndexException("Failed to find the base file for partition: " + partitionPath - + ", fileId: " + fileId); - } - fileIDBaseFileMap.put(fileId, baseFile.get()); - } - fileToKeysMap.computeIfAbsent(Pair.of(partitionPath, fileIDBaseFileMap.get(fileId).getFileName()), - k -> new ArrayList<>()).add(entry._2); - if (fileToKeysMap.size() > BLOOM_FILTER_CHECK_MAX_FILE_COUNT_PER_BATCH) { - break; - } - } - if (fileToKeysMap.isEmpty()) { - return Collections.emptyList(); - } - - List> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet()); - Map, BloomFilter> fileToBloomFilterMap = - hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList); - - final AtomicInteger totalKeys = new AtomicInteger(0); - fileToKeysMap.forEach((partitionPathFileNamePair, hoodieKeyList) -> { - final String partitionPath = partitionPathFileNamePair.getLeft(); - final String fileName = partitionPathFileNamePair.getRight(); - final String fileId = FSUtils.getFileId(fileName); - ValidationUtils.checkState(!fileId.isEmpty()); - - if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) { - throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair); - } - final BloomFilter fileBloomFilter = fileToBloomFilterMap.get(partitionPathFileNamePair); - - List candidateRecordKeys = new ArrayList<>(); - hoodieKeyList.forEach(hoodieKey -> { - totalKeys.incrementAndGet(); - if (fileBloomFilter.mightContain(hoodieKey.getRecordKey())) { - candidateRecordKeys.add(hoodieKey.getRecordKey()); - } - }); - - final HoodieBaseFile dataFile = fileIDBaseFileMap.get(fileId); - List matchingKeys = - HoodieIndexUtils.filterKeysFromFile(new Path(dataFile.getPath()), candidateRecordKeys, - hoodieTable.getHadoopConf()); - LOG.debug( - String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", - hoodieKeyList.size(), candidateRecordKeys.size(), - candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size())); - - resultList.add(new HoodieKeyLookupResult(fileId, partitionPath, dataFile.getCommitTime(), matchingKeys)); - }); - return resultList; - } - - @Override - protected void end() { - } - } -} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java index 5736024dc2455..265b0507768ff 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java @@ -19,30 +19,47 @@ package org.apache.hudi.index.bloom; -import org.apache.hudi.common.data.HoodieData; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.data.HoodiePairData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecordLocation; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.common.util.hash.FileIndexID; +import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.io.HoodieKeyLookupResult; import org.apache.hudi.table.HoodieTable; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import scala.Tuple2; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import scala.Tuple2; - +import static org.apache.hudi.metadata.HoodieMetadataPayload.getBloomFilterIndexKey; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex; import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS; /** @@ -55,8 +72,7 @@ public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper { private static final SparkHoodieBloomIndexHelper SINGLETON_INSTANCE = new SparkHoodieBloomIndexHelper(); - private SparkHoodieBloomIndexHelper() { - } + private SparkHoodieBloomIndexHelper() {} public static SparkHoodieBloomIndexHelper getInstance() { return SINGLETON_INSTANCE; @@ -66,42 +82,92 @@ public static SparkHoodieBloomIndexHelper getInstance() { public HoodiePairData findMatchingFilesForRecordKeys( HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable, HoodiePairData partitionRecordKeyPairs, - HoodieData> fileComparisonPairs, + HoodiePairData fileComparisonPairs, Map> partitionToFileInfo, Map recordsPerPartition) { - JavaRDD> fileComparisonsRDD = - HoodieJavaRDD.getJavaRDD(fileComparisonPairs) - .map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight())); - int inputParallelism = HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).partitions().size(); - int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism()); - LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${" - + config.getBloomIndexParallelism() + "}"); + int inputParallelism = HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).getNumPartitions(); + int configuredBloomIndexParallelism = config.getBloomIndexParallelism(); + // NOTE: Target parallelism could be overridden by the config + int targetParallelism = + configuredBloomIndexParallelism > 0 ? configuredBloomIndexParallelism : inputParallelism; + + LOG.info(String.format("Input parallelism: %d, Index parallelism: %d", inputParallelism, targetParallelism)); + + JavaPairRDD fileComparisonsRDD = HoodieJavaRDD.getJavaRDD(fileComparisonPairs); JavaRDD> keyLookupResultRDD; + if (config.getBloomIndexUseMetadata() && hoodieTable.getMetaClient().getTableConfig().getMetadataPartitions() .contains(BLOOM_FILTERS.getPartitionPath())) { - // Step 1: Sort by file id - JavaRDD> sortedFileIdAndKeyPairs = - fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism); + SerializableConfiguration hadoopConf = new SerializableConfiguration(hoodieTable.getHadoopConf()); + + HoodieTableFileSystemView baseFileOnlyView = + getBaseFileOnlyView(hoodieTable, partitionToFileInfo.keySet()); + + Broadcast baseFileOnlyViewBroadcast = + ((HoodieSparkEngineContext) context).getJavaSparkContext().broadcast(baseFileOnlyView); + + // When leveraging MT we're aiming for following goals: + // - (G1) All requests to MT are made in batch (ie we're trying to fetch all the values + // for corresponding keys at once) + // - (G2) Each task reads no more than just _one_ file-group from the MT Bloom Filters + // partition + // + // Ta achieve G2, following invariant have to be maintained: Spark partitions have to be + // affine w/ Metadata Table's file-groups, meaning that each Spark partition holds records + // belonging to one and only file-group in MT Bloom Filters partition. To provide for that + // we need to make sure + // - Spark's used [[Partitioner]] employs same hashing function as Metadata Table (as well + // as being applied to the same keys as the MT one) + // - Make sure that # of partitions is congruent to the # of file-groups (ie number of Spark + // partitions is a multiple of the # of the file-groups). + // + // Last provision is necessary, so that for every key it's the case that + // + // (hash(key) % N) % M = hash(key) % M, iff N % M = 0 + // + // Let's take an example of N = 8 and M = 4 (default # of file-groups in Bloom Filter + // partition). In that case Spark partitions for which `hash(key) % N` will be either 0 + // or 4, will map to the same (first) file-group in MT + int bloomFilterPartitionFileGroupCount = + config.getMetadataConfig().getBloomFilterIndexFileGroupCount(); + int adjustedTargetParallelism = + targetParallelism % bloomFilterPartitionFileGroupCount == 0 + ? targetParallelism + // NOTE: We add 1 to make sure parallelism a) value always stays positive and b) + // {@code targetParallelism <= adjustedTargetParallelism} + : (targetParallelism / bloomFilterPartitionFileGroupCount + 1) * bloomFilterPartitionFileGroupCount; + + AffineBloomIndexFileGroupPartitioner partitioner = + new AffineBloomIndexFileGroupPartitioner(baseFileOnlyViewBroadcast, adjustedTargetParallelism); + + // First, we need to repartition and sort records using [[AffineBloomIndexFileGroupPartitioner]] + // to make sure every Spark task accesses no more than just a single file-group in MT (allows + // us to achieve G2). + // + // NOTE: Sorting records w/in individual partitions is required to make sure that we cluster + // together keys co-located w/in the MT files (sorted by keys) + keyLookupResultRDD = fileComparisonsRDD.repartitionAndSortWithinPartitions(partitioner) + .mapPartitionsToPair(new HoodieMetadataBloomFilterProbingFunction(baseFileOnlyViewBroadcast, hoodieTable)) + // Second, we use [[HoodieFileProbingFunction]] to open actual file and check whether it + // contains the records with candidate keys that were filtered in by the Bloom Filter + .mapPartitions(new HoodieFileProbingFunction(baseFileOnlyViewBroadcast, hadoopConf), true); - // Step 2: Use bloom filter to filter and the actual log file to get the record location - keyLookupResultRDD = sortedFileIdAndKeyPairs.mapPartitionsWithIndex( - new HoodieMetadataBloomIndexCheckFunction(hoodieTable), true); } else if (config.useBloomIndexBucketizedChecking()) { - Map comparisonsPerFileGroup = computeComparisonsPerFileGroup( + Map comparisonsPerFileGroup = computeComparisonsPerFileGroup( config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context); - Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup, + Partitioner partitioner = new BucketizedBloomCheckPartitioner(targetParallelism, comparisonsPerFileGroup, config.getBloomIndexKeysPerBucket()); - keyLookupResultRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t)) + keyLookupResultRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2), t)) .repartitionAndSortWithinPartitions(partitioner) .map(Tuple2::_2) - .mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true); + .mapPartitions(new HoodieSparkBloomIndexCheckFunction(hoodieTable, config), true); } else { - keyLookupResultRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism) - .mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true); + keyLookupResultRDD = fileComparisonsRDD.sortByKey(true, targetParallelism) + .mapPartitions(new HoodieSparkBloomIndexCheckFunction(hoodieTable, config), true); } return HoodieJavaPairRDD.of(keyLookupResultRDD.flatMap(List::iterator) @@ -115,27 +181,124 @@ public HoodiePairData findMatchingFilesForRecor /** * Compute the estimated number of bloom filter comparisons to be performed on each file group. */ - private Map computeComparisonsPerFileGroup( + private Map computeComparisonsPerFileGroup( final HoodieWriteConfig config, final Map recordsPerPartition, final Map> partitionToFileInfo, - final JavaRDD> fileComparisonsRDD, + final JavaPairRDD fileComparisonsRDD, final HoodieEngineContext context) { - Map fileToComparisons; + Map fileToComparisons; if (config.getBloomIndexPruneByRanges()) { // we will just try exploding the input and then count to determine comparisons // FIX(vc): Only do sampling here and extrapolate? context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files: " + config.getTableName()); - fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey(); + fileToComparisons = fileComparisonsRDD.countByKey(); } else { fileToComparisons = new HashMap<>(); - partitionToFileInfo.forEach((key, value) -> { - for (BloomIndexFileInfo fileInfo : value) { + partitionToFileInfo.forEach((partitionPath, fileInfos) -> { + for (BloomIndexFileInfo fileInfo : fileInfos) { // each file needs to be compared against all the records coming into the partition - fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(key)); + fileToComparisons.put( + new HoodieFileGroupId(partitionPath, fileInfo.getFileId()), recordsPerPartition.get(partitionPath)); } }); } return fileToComparisons; } + + private static HoodieTableFileSystemView getBaseFileOnlyView(HoodieTable hoodieTable, Collection partitionPaths) { + try { + List fullPartitionPaths = partitionPaths.stream() + .map(partitionPath -> + String.format("%s/%s", hoodieTable.getMetaClient().getBasePathV2(), partitionPath)) + .collect(Collectors.toList()); + + FileStatus[] allFiles = + hoodieTable.getMetadataTable().getAllFilesInPartitions(fullPartitionPaths).values().stream() + .flatMap(Arrays::stream) + .toArray(FileStatus[]::new); + + return new HoodieTableFileSystemView(hoodieTable.getMetaClient(), hoodieTable.getActiveTimeline(), allFiles); + } catch (IOException e) { + LOG.error(String.format("Failed to fetch all files for partitions (%s)", partitionPaths)); + throw new HoodieIOException("Failed to fetch all files for partitions", e); + } + } + + static class AffineBloomIndexFileGroupPartitioner extends Partitioner { + + private final Broadcast latestBaseFilesBroadcast; + + // TODO(HUDI-5619) remove when addressed + private final Map> cachedLatestBaseFileNames = + new HashMap<>(16); + + private final int targetPartitions; + + AffineBloomIndexFileGroupPartitioner(Broadcast baseFileOnlyViewBroadcast, + int targetPartitions) { + this.targetPartitions = targetPartitions; + this.latestBaseFilesBroadcast = baseFileOnlyViewBroadcast; + } + + @Override + public int numPartitions() { + return targetPartitions; + } + + @Override + public int getPartition(Object key) { + HoodieFileGroupId partitionFileGroupId = (HoodieFileGroupId) key; + String partitionPath = partitionFileGroupId.getPartitionPath(); + String fileGroupId = partitionFileGroupId.getFileId(); + + /* + // TODO(HUDI-5619) uncomment when addressed + String baseFileName = + latestBaseFilesBroadcast.getValue() + .getLatestBaseFile(partitionPath, fileGroupId) + .orElseThrow(() -> new HoodieException( + String.format("File from file-group (%s) not found in partition path (%s)", fileGroupId, partitionPath))) + .getFileName(); + */ + + // NOTE: This is a workaround to alleviate performance impact of needing to process whole + // partition for every file-group being looked up. + // See HUDI-5619 for more details + String baseFileName = cachedLatestBaseFileNames.computeIfAbsent(partitionPath, ignored -> + latestBaseFilesBroadcast.getValue() + .getLatestBaseFiles(partitionPath) + .collect( + Collectors.toMap(HoodieBaseFile::getFileId, BaseFile::getFileName) + ) + ) + .get(fileGroupId); + + if (baseFileName == null) { + throw new HoodieException( + String.format("File from file-group (%s) not found in partition path (%s)", fileGroupId, partitionPath)); + } + + String bloomIndexEncodedKey = + getBloomFilterIndexKey(new PartitionIndexID(partitionPath), new FileIndexID(baseFileName)); + + // NOTE: It's crucial that [[targetPartitions]] be congruent w/ the number of + // actual file-groups in the Bloom Index in MT + return mapRecordKeyToFileGroupIndex(bloomIndexEncodedKey, targetPartitions); + } + } + + public static class HoodieSparkBloomIndexCheckFunction extends HoodieBloomIndexCheckFunction> + implements FlatMapFunction>, List> { + + public HoodieSparkBloomIndexCheckFunction(HoodieTable hoodieTable, + HoodieWriteConfig config) { + super(hoodieTable, config, t -> t._1, t -> t._2); + } + + @Override + public Iterator> call(Iterator> fileGroupIdRecordKeyPairIterator) { + return new LazyKeyCheckIterator(fileGroupIdRecordKeyPairIterator); + } + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java index cbc822f9acf34..6ad15b4e92d80 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java @@ -40,6 +40,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.table.HoodieTable; @@ -56,11 +57,16 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.TreeSet; import java.util.function.Predicate; import java.util.stream.Collectors; import scala.Tuple2; +import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASHING_METADATA_COMMIT_FILE_SUFFIX; +import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX; +import static org.apache.hudi.common.model.HoodieConsistentHashingMetadata.getTimestampFromFile; + /** * Consistent hashing bucket index implementation, with auto-adjust bucket number. * NOTE: bucket resizing is triggered by clustering. @@ -189,29 +195,64 @@ public HoodieConsistentHashingMetadata loadOrCreateMetadata(HoodieTable table, S */ public static Option loadMetadata(HoodieTable table, String partition) { Path metadataPath = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), partition); - + Path partitionPath = FSUtils.getPartitionPath(table.getMetaClient().getBasePathV2(), partition); try { - FileStatus[] metaFiles = table.getMetaClient().getFs().listStatus(metadataPath); - final HoodieTimeline completedCommits = table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(); - Predicate metaFilePredicate = fileStatus -> { + Predicate hashingMetaCommitFilePredicate = fileStatus -> { String filename = fileStatus.getPath().getName(); - if (!filename.contains(HoodieConsistentHashingMetadata.HASHING_METADATA_FILE_SUFFIX)) { - return false; - } - String timestamp = HoodieConsistentHashingMetadata.getTimestampFromFile(filename); - return completedCommits.containsInstant(timestamp) || timestamp.equals(HoodieTimeline.INIT_INSTANT_TS); + return filename.contains(HoodieConsistentHashingMetadata.HASHING_METADATA_COMMIT_FILE_SUFFIX); }; - - // Get a valid hashing metadata with the largest (latest) timestamp - FileStatus metaFile = Arrays.stream(metaFiles).filter(metaFilePredicate) - .max(Comparator.comparing(a -> a.getPath().getName())).orElse(null); - - if (metaFile == null) { - return Option.empty(); + Predicate hashingMetadataFilePredicate = fileStatus -> { + String filename = fileStatus.getPath().getName(); + return filename.contains(HASHING_METADATA_FILE_SUFFIX); + }; + final FileStatus[] metaFiles = table.getMetaClient().getFs().listStatus(metadataPath); + final TreeSet commitMetaTss = Arrays.stream(metaFiles).filter(hashingMetaCommitFilePredicate) + .map(commitFile -> HoodieConsistentHashingMetadata.getTimestampFromFile(commitFile.getPath().getName())) + .sorted() + .collect(Collectors.toCollection(TreeSet::new)); + final FileStatus[] hashingMetaFiles = Arrays.stream(metaFiles).filter(hashingMetadataFilePredicate) + .sorted(Comparator.comparing(f -> f.getPath().getName())) + .toArray(FileStatus[]::new); + // max committed metadata file + final String maxCommitMetaFileTs = commitMetaTss.isEmpty() ? null : commitMetaTss.last(); + // max updated metadata file + FileStatus maxMetadataFile = hashingMetaFiles.length > 0 ? hashingMetaFiles[hashingMetaFiles.length - 1] : null; + // If single file present in metadata and if its default file return it + if (maxMetadataFile != null && HoodieConsistentHashingMetadata.getTimestampFromFile(maxMetadataFile.getPath().getName()).equals(HoodieTimeline.INIT_INSTANT_TS)) { + return loadMetadataFromGivenFile(table, maxMetadataFile); } + // if max updated metadata file and committed metadata file are same then return + if (maxCommitMetaFileTs != null && maxMetadataFile != null + && maxCommitMetaFileTs.equals(HoodieConsistentHashingMetadata.getTimestampFromFile(maxMetadataFile.getPath().getName()))) { + return loadMetadataFromGivenFile(table, maxMetadataFile); + } + HoodieTimeline completedCommits = table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(); - byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath())); - return Option.of(HoodieConsistentHashingMetadata.fromBytes(content)); + // fix the in-consistency between un-committed and committed hashing metadata files. + List fixed = new ArrayList<>(); + Arrays.stream(hashingMetaFiles).forEach(hashingMetaFile -> { + Path path = hashingMetaFile.getPath(); + String timestamp = HoodieConsistentHashingMetadata.getTimestampFromFile(path.getName()); + if (maxCommitMetaFileTs != null && timestamp.compareTo(maxCommitMetaFileTs) <= 0) { + // only fix the metadata with greater timestamp than max committed timestamp + return; + } + boolean isRehashingCommitted = completedCommits.containsInstant(timestamp) || timestamp.equals(HoodieTimeline.INIT_INSTANT_TS); + if (isRehashingCommitted) { + if (!commitMetaTss.contains(timestamp)) { + try { + createCommitMarker(table, path, partitionPath); + } catch (IOException e) { + throw new HoodieIOException("Exception while creating marker file " + path.getName() + " for partition " + partition, e); + } + } + fixed.add(hashingMetaFile); + } else if (recommitMetadataFile(table, hashingMetaFile, partition)) { + fixed.add(hashingMetaFile); + } + }); + + return fixed.isEmpty() ? Option.empty() : loadMetadataFromGivenFile(table, fixed.get(fixed.size() - 1)); } catch (FileNotFoundException e) { return Option.empty(); } catch (IOException e) { @@ -271,8 +312,83 @@ public Option getRecordLocation(HoodieKey key) { } LOG.error("Consistent hashing node has no file group, partition: " + partitionPath + ", meta: " - + partitionToIdentifier.get(partitionPath).getMetadata().getFilename() + ", record_key: " + key.toString()); + + partitionToIdentifier.get(partitionPath).getMetadata().getFilename() + ", record_key: " + key.toString()); throw new HoodieIndexException("Failed to getBucket as hashing node has no file group"); } } + + /*** + * Creates commit marker corresponding to hashing metadata file after post commit clustering operation. + * @param table hoodie table + * @param fileStatus file for which commit marker should be created + * @param partitionPath partition path the file belongs to + * @throws IOException + */ + private static void createCommitMarker(HoodieTable table, Path fileStatus, Path partitionPath) throws IOException { + HoodieWrapperFileSystem fs = table.getMetaClient().getFs(); + Path fullPath = new Path(partitionPath, getTimestampFromFile(fileStatus.getName()) + HASHING_METADATA_COMMIT_FILE_SUFFIX); + if (fs.exists(fullPath)) { + return; + } + FileIOUtils.createFileInPath(fs, fullPath, Option.of(StringUtils.EMPTY_STRING.getBytes())); + } + + /*** + * Loads consistent hashing metadata of table from the given meta file + * @param table hoodie table + * @param metaFile hashing metadata file + * @return HoodieConsistentHashingMetadata object + */ + private static Option loadMetadataFromGivenFile(HoodieTable table, FileStatus metaFile) { + try { + if (metaFile == null) { + return Option.empty(); + } + byte[] content = FileIOUtils.readAsByteArray(table.getMetaClient().getFs().open(metaFile.getPath())); + return Option.of(HoodieConsistentHashingMetadata.fromBytes(content)); + } catch (FileNotFoundException e) { + return Option.empty(); + } catch (IOException e) { + LOG.error("Error when loading hashing metadata, for path: " + metaFile.getPath().getName(), e); + throw new HoodieIndexException("Error while loading hashing metadata", e); + } + } + + /*** + * COMMIT MARKER RECOVERY JOB. + * If particular hashing metadta file doesn't have commit marker then there could be a case where clustering is done but post commit marker + * creation operation failed. In this case this method will check file group id from consistent hashing metadata against storage base file group ids. + * if one of the file group matches then we can conclude that this is the latest metadata file. + * Note : we will end up calling this method if there is no marker file and no replace commit on active timeline, if replace commit is not present on + * active timeline that means old file group id's before clustering operation got cleaned and only new file group id's of current clustering operation + * are present on the disk. + * @param table hoodie table + * @param metaFile metadata file on which sync check needs to be performed + * @param partition partition metadata file belongs to + * @return true if hashing metadata file is latest else false + */ + private static boolean recommitMetadataFile(HoodieTable table, FileStatus metaFile, String partition) { + Path partitionPath = FSUtils.getPartitionPath(table.getMetaClient().getBasePathV2(), partition); + String timestamp = getTimestampFromFile(metaFile.getPath().getName()); + if (table.getPendingCommitTimeline().containsInstant(timestamp)) { + return false; + } + Option hoodieConsistentHashingMetadataOption = loadMetadataFromGivenFile(table, metaFile); + if (!hoodieConsistentHashingMetadataOption.isPresent()) { + return false; + } + HoodieConsistentHashingMetadata hoodieConsistentHashingMetadata = hoodieConsistentHashingMetadataOption.get(); + + Predicate hoodieFileGroupIdPredicate = hoodieBaseFile -> hoodieConsistentHashingMetadata.getNodes().stream().anyMatch(node -> node.getFileIdPrefix().equals(hoodieBaseFile)); + if (table.getBaseFileOnlyView().getLatestBaseFiles(partition) + .map(fileIdPrefix -> FSUtils.getFileIdPfxFromFileId(fileIdPrefix.getFileId())).anyMatch(hoodieFileGroupIdPredicate)) { + try { + createCommitMarker(table, metaFile.getPath(), partitionPath); + return true; + } catch (IOException e) { + throw new HoodieIOException("Exception while creating marker file " + metaFile.getPath().getName() + " for partition " + partition, e); + } + } + return false; + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkBootstrapFileReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkBootstrapFileReader.java new file mode 100644 index 0000000000000..2f355d471b752 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkBootstrapFileReader.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.util.Option; + +import org.apache.spark.sql.catalyst.InternalRow; + +public class HoodieSparkBootstrapFileReader extends HoodieBootstrapFileReader { + + public HoodieSparkBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + super(skeletonFileReader, dataFileReader, partitionFields, partitionValues); + } + + @Override + protected void setPartitionField(int position, Object fieldValue, InternalRow row) { + if (row.isNullAt(position)) { + row.update(position, fieldValue); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java index 112981f9022dd..6c94c1c54d71d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java @@ -18,21 +18,23 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hudi.exception.HoodieIOException; +import org.apache.spark.sql.internal.SQLConf; import java.io.IOException; -import org.apache.spark.sql.internal.SQLConf; -public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory { +public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory { protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) { - conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), - SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); - conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), - SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); + conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(), SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()); + conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()); conf.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString()); + // Using string value of this conf to preserve compatibility across spark versions. + conf.setIfUnset("spark.sql.legacy.parquet.nanosAsLong", "false"); return new HoodieSparkParquetReader(conf, path); } @@ -43,4 +45,9 @@ protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) thr protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { throw new HoodieIOException("Not support read orc file"); } + + @Override + public HoodieFileReader newBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + return new HoodieSparkBootstrapFileReader(skeletonFileReader, dataFileReader, partitionFields, partitionValues); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java index 58ee01182c373..3c69f0ab5c2ec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java @@ -21,18 +21,11 @@ import org.apache.avro.Schema; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.spark.sql.catalyst.CatalystTypeConverters; import org.apache.spark.sql.catalyst.InternalRow; import java.io.IOException; import java.util.Properties; -import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD; -import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD; -import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD; -import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD; -import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD; - public interface HoodieSparkFileWriter extends HoodieFileWriter { boolean canWrite(); @@ -51,14 +44,4 @@ default void write(String recordKey, HoodieRecord record, Schema schema, Propert default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException { writeRowWithMetadata(key, (InternalRow) record.getData()); } - - default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName) { - String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); - row.update(COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime)); - row.update(COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId)); - row.update(RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey())); - row.update(PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath())); - row.update(FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName)); - return row; - } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java index 320217aff1d4a..e24f03a4cb453 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java @@ -122,7 +122,8 @@ public Schema getSchema() { // and therefore if we convert to Avro directly we'll lose logical type-info. MessageType messageType = ((ParquetUtils) parquetUtils).readSchema(conf, path); StructType structType = new ParquetToSparkSchemaConverter(conf).convert(messageType); - return SparkAdapterSupport$.MODULE$.sparkAdapter().getAvroSchemaConverters() + return SparkAdapterSupport$.MODULE$.sparkAdapter() + .getAvroSchemaConverters() .toAvroType(structType, true, messageType.getName(), StringUtils.EMPTY_STRING); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java index 3b4a86502d280..d601e6ded3e12 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java @@ -21,22 +21,32 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.io.storage.row.HoodieRowParquetConfig; import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; +import java.util.function.Function; + +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD; +import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD; public class HoodieSparkParquetWriter extends HoodieBaseParquetWriter implements HoodieSparkFileWriter { - // TODO: better code reuse - private final String fileName; - private final String instantTime; - private final TaskContextSupplier taskContextSupplier; + private final UTF8String fileName; + private final UTF8String instantTime; + private final boolean populateMetaFields; + private final HoodieRowParquetWriteSupport writeSupport; + private final Function seqIdGenerator; + public HoodieSparkParquetWriter(Path file, HoodieRowParquetConfig parquetConfig, String instantTime, @@ -44,19 +54,23 @@ public HoodieSparkParquetWriter(Path file, boolean populateMetaFields) throws IOException { super(file, parquetConfig); this.writeSupport = parquetConfig.getWriteSupport(); - this.fileName = file.getName(); - this.instantTime = instantTime; - this.taskContextSupplier = taskContextSupplier; + this.fileName = UTF8String.fromString(file.getName()); + this.instantTime = UTF8String.fromString(instantTime); this.populateMetaFields = populateMetaFields; + this.seqIdGenerator = recordIndex -> { + Integer partitionId = taskContextSupplier.getPartitionIdSupplier().get(); + return HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex); + }; } @Override public void writeRowWithMetadata(HoodieKey key, InternalRow row) throws IOException { if (populateMetaFields) { - prepRecordWithMetadata(key, row, instantTime, - taskContextSupplier.getPartitionIdSupplier().get(), getWrittenRecordCount(), fileName); + UTF8String recordKey = UTF8String.fromString(key.getRecordKey()); + updateRecordMetadata(row, recordKey, key.getPartitionPath(), getWrittenRecordCount()); + super.write(row); - writeSupport.add(UTF8String.fromString(key.getRecordKey())); + writeSupport.add(recordKey); } else { super.write(row); } @@ -74,4 +88,16 @@ public void writeRow(String recordKey, InternalRow row) throws IOException { public void close() throws IOException { super.close(); } + + protected void updateRecordMetadata(InternalRow row, + UTF8String recordKey, + String partitionPath, + long recordCount) { + row.update(COMMIT_TIME_METADATA_FIELD.ordinal(), instantTime); + row.update(COMMIT_SEQNO_METADATA_FIELD.ordinal(), UTF8String.fromString(seqIdGenerator.apply(recordCount))); + row.update(RECORD_KEY_METADATA_FIELD.ordinal(), recordKey); + // TODO set partition path in ctor + row.update(PARTITION_PATH_METADATA_FIELD.ordinal(), UTF8String.fromString(partitionPath)); + row.update(FILENAME_METADATA_FIELD.ordinal(), fileName); + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java index 40154d8675cff..b95f1b1ef743b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java @@ -480,7 +480,7 @@ private Object[] getNestedFieldValues(InternalRow row, HoodieUnsafeRowUtils.Nest private HoodieUnsafeRowUtils.NestedFieldPath[] resolveNestedFieldPaths(List fieldPaths, StructType schema, boolean returnNull) { try { return fieldPaths.stream() - .map(fieldPath -> HoodieUnsafeRowUtils$.MODULE$.composeNestedFieldPath(schema, fieldPath)) + .map(fieldPath -> HoodieUnsafeRowUtils$.MODULE$.composeNestedFieldPath(schema, fieldPath).get()) .toArray(HoodieUnsafeRowUtils.NestedFieldPath[]::new); } catch (Exception e) { if (returnNull) { 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 81526c25bccde..1cb0d73acbe89 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 @@ -19,11 +19,11 @@ package org.apache.hudi.metadata; import org.apache.hudi.client.SparkRDDWriteClient; -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.metrics.Registry; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; @@ -35,7 +35,6 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; -import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.avro.specific.SpecificRecordBase; @@ -49,6 +48,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER; + public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter { private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class); @@ -73,8 +74,18 @@ public static HoodieTableMetadataWriter create(Co HoodieEngineContext context, Option actionMetadata, Option inflightInstantTimestamp) { - return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, - inflightInstantTimestamp); + return new SparkHoodieBackedTableMetadataWriter( + conf, writeConfig, EAGER, context, actionMetadata, inflightInstantTimestamp); + } + + public static HoodieTableMetadataWriter create(Configuration conf, + HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + HoodieEngineContext context, + Option actionMetadata, + Option inflightInstantTimestamp) { + return new SparkHoodieBackedTableMetadataWriter( + conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inflightInstantTimestamp); } public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, @@ -84,10 +95,11 @@ public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteCo SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, HoodieEngineContext engineContext, Option actionMetadata, Option inflightInstantTimestamp) { - super(hadoopConf, writeConfig, engineContext, actionMetadata, inflightInstantTimestamp); + super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inflightInstantTimestamp); } @Override @@ -133,9 +145,11 @@ protected void commit(String instantTime, Map preppedRecords = prepRecords(partitionRecordsMap); JavaRDD preppedRecordRDD = HoodieJavaRDD.getJavaRDD(preppedRecords); + engineContext.setJobStatus(this.getClass().getName(), "Committing " + instantTime + " to metadata table " + metadataWriteConfig.getTableName()); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig)) { // rollback partially failed writes if any. - if (writeClient.rollbackFailedWrites()) { + if (dataWriteConfig.getFailedWritesCleanPolicy().isEager() + && writeClient.rollbackFailedWrites()) { metadataMetaClient = HoodieTableMetaClient.reload(metadataMetaClient); } if (canTriggerTableService) { @@ -168,12 +182,7 @@ protected void commit(String instantTime, Map statuses = writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect(); - statuses.forEach(writeStatus -> { - if (writeStatus.hasErrors()) { - throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime); - } - }); + writeClient.upsertPreppedRecords(preppedRecordRDD, instantTime).collect(); // reload timeline metadataMetaClient.reloadActiveTimeline(); 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 112deccf8df34..d6796a7a4d489 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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -89,14 +90,17 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con * @return instance of {@link HoodieTableMetadataWriter} */ @Override - public Option getMetadataWriter(String triggeringInstantTimestamp, - Option actionMetadata) { + protected Option getMetadataWriter( + String triggeringInstantTimestamp, + HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, + Option actionMetadata) { if (config.isMetadataTableEnabled()) { // Create the metadata table writer. First time after the upgrade this creation might trigger // metadata table bootstrapping. Bootstrapping process could fail and checking the table // existence after the creation is needed. final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( - context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp)); + context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context, + actionMetadata, Option.of(triggeringInstantTimestamp)); // even with metadata enabled, some index could have been disabled // delete metadata partitions corresponding to such indexes deleteMetadataIndexIfNecessary(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java index 0bc15fa2106a5..3e63d0bb22c09 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/BaseBootstrapMetadataHandler.java @@ -34,7 +34,6 @@ import org.apache.hudi.table.HoodieTable; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.avro.AvroReadSupport; import java.io.IOException; import java.util.List; @@ -62,10 +61,10 @@ public BootstrapWriteStatus runMetadataBootstrap(String srcPartitionPath, String .map(HoodieAvroUtils::getRootLevelFieldName) .collect(Collectors.toList()); Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, recordKeyColumns); - LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); - AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); - AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); - executeBootstrap(bootstrapHandle, sourceFilePath, keyGenerator, partitionPath, avroSchema); + + LOG.info("Schema to be used for reading record keys: " + recordKeySchema); + + executeBootstrap(bootstrapHandle, sourceFilePath, keyGenerator, partitionPath, recordKeySchema); } catch (Exception e) { throw new HoodieException(e.getMessage(), e); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java index d7a4a2b52abf7..d2f7da4edcb48 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java @@ -18,7 +18,6 @@ package org.apache.hudi.table.action.bootstrap; -import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; @@ -47,6 +46,8 @@ import java.io.IOException; +import static org.apache.hudi.io.HoodieBootstrapHandle.METADATA_BOOTSTRAP_RECORD_SCHEMA; + class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { private static final Logger LOG = LogManager.getLogger(OrcBootstrapMetadataHandler.class); @@ -70,12 +71,12 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, Path so } BoundedInMemoryExecutor wrapper = null; Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf())); - TypeDescription orcSchema = orcReader.getSchema(); + TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(avroSchema); try (RecordReader reader = orcReader.rows(new Reader.Options(table.getHadoopConf()).schema(orcSchema))) { wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), new OrcReaderIterator(reader, avroSchema, orcSchema), new BootstrapRecordConsumer(bootstrapHandle), inp -> { String recKey = keyGenerator.getKey(inp).getRecordKey(); - GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); + GenericRecord gr = new GenericData.Record(METADATA_BOOTSTRAP_RECORD_SCHEMA); gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java index d008d7cf9fd96..e5944225750d0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java @@ -18,11 +18,15 @@ package org.apache.hudi.table.action.bootstrap; -import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; +import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieSparkRecord; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; import org.apache.hudi.config.HoodieWriteConfig; @@ -35,19 +39,24 @@ import org.apache.avro.Schema; import org.apache.hadoop.fs.Path; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.HoodieInternalRowUtils$; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; import java.io.IOException; -import java.util.Properties; +import java.util.function.Function; + +import static org.apache.hudi.io.HoodieBootstrapHandle.METADATA_BOOTSTRAP_RECORD_SCHEMA; class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler { - private static final Logger LOG = LogManager.getLogger(ParquetBootstrapMetadataHandler.class); public ParquetBootstrapMetadataHandler(HoodieWriteConfig config, HoodieTable table, HoodieFileStatus srcFileStatus) { super(config, table, srcFileStatus); @@ -62,31 +71,29 @@ Schema getAvroSchema(Path sourceFilePath) throws IOException { } @Override - void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, - Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception { + protected void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, + Path sourceFilePath, + KeyGeneratorInterface keyGenerator, + String partitionPath, + Schema schema) throws Exception { BoundedInMemoryExecutor wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()) + HoodieRecordMerger recordMerger = table.getConfig().getRecordMerger(); + + HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(recordMerger.getRecordType()) .getFileReader(table.getHadoopConf(), sourceFilePath); try { + Function transformer = record -> { + String recordKey = record.getRecordKey(schema, Option.of(keyGenerator)); + return createNewMetadataBootstrapRecord(recordKey, partitionPath, recordMerger.getRecordType()) + // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific + // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of + // it since these records will be inserted into the queue later. + .copy(); + }; + wrapper = new BoundedInMemoryExecutor(config.getWriteBufferLimitBytes(), - reader.getRecordIterator(), new BootstrapRecordConsumer(bootstrapHandle), record -> { - try { - String recKey = record.getRecordKey(reader.getSchema(), Option.of(keyGenerator)); - // NOTE: Record have to be cloned here to make sure if it holds low-level engine-specific - // payload pointing into a shared, mutable (underlying) buffer we get a clean copy of - // it since these records will be inserted into the queue later. - HoodieRecord hoodieRecord = record - .rewriteRecord(reader.getSchema(), config.getProps(), HoodieAvroUtils.RECORD_KEY_SCHEMA) - .copy(); - MetadataValues metadataValues = new MetadataValues().setRecordKey(recKey); - return hoodieRecord - .updateMetadataValues(HoodieAvroUtils.RECORD_KEY_SCHEMA, new Properties(), metadataValues) - .newInstance(new HoodieKey(recKey, partitionPath)); - } catch (IOException e) { - LOG.error("Unable to overrideMetadataFieldValue", e); - return null; - } - }, table.getPreExecuteRunnable()); + reader.getRecordIterator(schema), new BootstrapRecordConsumer(bootstrapHandle), transformer, table.getPreExecuteRunnable()); + wrapper.execute(); } catch (Exception e) { throw new HoodieException(e); @@ -99,5 +106,31 @@ void executeBootstrap(HoodieBootstrapHandle bootstrapHandle, bootstrapHandle.close(); } } + + private HoodieRecord createNewMetadataBootstrapRecord(String recordKey, String partitionPath, HoodieRecord.HoodieRecordType recordType) { + HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath); + switch (recordType) { + case AVRO: + GenericRecord avroRecord = new GenericData.Record(METADATA_BOOTSTRAP_RECORD_SCHEMA); + avroRecord.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recordKey); + BootstrapRecordPayload payload = new BootstrapRecordPayload(avroRecord); + return new HoodieAvroRecord<>(hoodieKey, payload); + + case SPARK: + StructType schema = HoodieInternalRowUtils$.MODULE$.getCachedSchema(METADATA_BOOTSTRAP_RECORD_SCHEMA); + UnsafeProjection unsafeProjection = HoodieInternalRowUtils$.MODULE$.getCachedUnsafeProjection(schema, schema); + + GenericInternalRow row = new GenericInternalRow(METADATA_BOOTSTRAP_RECORD_SCHEMA.getFields().size()); + row.update(HoodieRecord.RECORD_KEY_META_FIELD_ORD, UTF8String.fromString(recordKey)); + + UnsafeRow unsafeRow = unsafeProjection.apply(row); + + return new HoodieSparkRecord(hoodieKey, unsafeRow,false); + + default: + throw new UnsupportedOperationException(String.format("Record type %s is not supported yet!", recordType)); + } + + } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index e3524f4d70959..4ddd481b32831 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -368,9 +368,10 @@ private HoodieData runMetadataBootstrap(List getMetadataHandler(config, table, partitionFsPair.getRight().getRight()).runMetadataBootstrap(partitionFsPair.getLeft(), - partitionFsPair.getRight().getLeft(), keyGenerator)); + partitionFsPair.getRight().getLeft(), keyGenerator)); } @Override 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 231825875972b..a246a7150c932 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; @@ -42,6 +43,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). */ @@ -57,6 +61,7 @@ public class SparkBucketIndexPartitioner extends * 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. @@ -84,6 +89,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) { @@ -106,6 +113,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 49134d604d2d0..b45a691fbad83 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 @@ -20,6 +20,7 @@ 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.WriteOperationType; @@ -59,6 +60,8 @@ public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, @Override public HoodieWriteMetadata> execute() { + DeletePartitionUtils.checkForPendingTableServiceActions(table, partitions); + try { HoodieTimer timer = HoodieTimer.start(); 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 ee3b31cc577ba..b265b32da8edc 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 @@ -34,6 +34,7 @@ import org.apache.spark.Partitioner; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -85,4 +86,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 57668aada1ca7..5d82253da861e 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/util/HoodieSparkRecordUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java deleted file mode 100644 index 9a4aaa1dbc585..0000000000000 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.util; - -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; - -import org.apache.spark.sql.HoodieInternalRowUtils; -import org.apache.spark.sql.HoodieUnsafeRowUtils; -import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructType; - -public class HoodieSparkRecordUtils { - - public static Object getValue(StructType structType, String fieldName, InternalRow row) { - NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName); - return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList); - } - - /** - * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing. - * - * @param row The record - * @param fieldName The field name - * @return the string form of the field or empty if the schema does not contain the field name or the value is null - */ - public static Option getNullableValAsString(StructType structType, InternalRow row, String fieldName) { - String fieldVal = !HoodieInternalRowUtils.existField(structType, fieldName) - ? null : StringUtils.objToString(getValue(structType, fieldName, row)); - return Option.ofNullable(fieldVal); - } - - /** - * Gets record column values into one object. - * - * @param row InternalRow. - * @param columns Names of the columns to get values. - * @param structType {@link StructType} instance. - * @return Column value if a single column, or concatenated String values by comma. - */ - public static Object[] getRecordColumnValues(InternalRow row, - String[] columns, - StructType structType, boolean consistentLogicalTimestampEnabled) { - Object[] objects = new Object[columns.length]; - for (int i = 0; i < objects.length; i++) { - NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, columns[i]); - Object value = HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList); - objects[i] = value; - } - return objects; - } -} diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala index 238fed526a18e..e239db1b5a581 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieDatasetBulkInsertHelper.scala @@ -25,6 +25,7 @@ import org.apache.hudi.common.engine.TaskContextSupplier import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.util.ReflectionUtils import org.apache.hudi.config.HoodieWriteConfig +import org.apache.hudi.exception.HoodieException import org.apache.hudi.index.SparkHoodieIndexFactory import org.apache.hudi.keygen.{BuiltinKeyGenerator, SparkKeyGeneratorInterface} import org.apache.hudi.table.action.commit.{BulkInsertDataInternalWriterHelper, ParallelismHelper} @@ -33,7 +34,7 @@ import org.apache.hudi.util.JFunction.toJavaSerializableFunctionUnchecked import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue} -import org.apache.spark.sql.HoodieUnsafeUtils.getOutputPartitioning +import org.apache.spark.sql.HoodieUnsafeUtils.getNumPartitions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} import org.apache.spark.sql.catalyst.plans.logical.Project @@ -44,7 +45,7 @@ import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters.{asScalaBufferConverter, seqAsJavaListConverter} object HoodieDatasetBulkInsertHelper - extends ParallelismHelper[DataFrame](toJavaSerializableFunctionUnchecked(df => getOutputPartitioning(df).numPartitions)) with Logging { + extends ParallelismHelper[DataFrame](toJavaSerializableFunctionUnchecked(df => getNumPartitions(df))) with Logging { /** * Prepares [[DataFrame]] for bulk-insert into Hudi table, taking following steps: @@ -175,6 +176,7 @@ object HoodieDatasetBulkInsertHelper val partitionPathMetaFieldOrd = schema.fieldIndex(HoodieRecord.PARTITION_PATH_METADATA_FIELD) // NOTE: Pre-combine field could be a nested field val preCombineFieldPath = composeNestedFieldPath(schema, preCombineFieldRef) + .getOrElse(throw new HoodieException(s"Pre-combine field $preCombineFieldRef is missing in $schema")) rdd.map { row => val rowKey = if (isGlobalIndex) { @@ -201,6 +203,17 @@ object HoodieDatasetBulkInsertHelper .values } + override protected def deduceShuffleParallelism(input: DataFrame, configuredParallelism: Int): Int = { + val deduceParallelism = super.deduceShuffleParallelism(input, configuredParallelism) + // NOTE: In case parallelism deduction failed to accurately deduce parallelism level of the + // incoming dataset we fallback to default parallelism level set for this Spark session + if (deduceParallelism > 0) { + deduceParallelism + } else { + input.sparkSession.sparkContext.defaultParallelism + } + } + private def dropPartitionColumns(df: DataFrame, config: HoodieWriteConfig): DataFrame = { val partitionPathFields = getPartitionPathFields(config).toSet val nestedPartitionPathFields = partitionPathFields.filter(f => f.contains('.')) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala index bd7d3647b2ea2..219746882b66e 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala @@ -20,17 +20,23 @@ package org.apache.hudi import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord +import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.hadoop.CachingPath import org.apache.spark.SPARK_VERSION +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone import org.apache.spark.sql.execution.SQLConfInjectingRDD +import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -52,9 +58,10 @@ private[hudi] trait SparkVersionsSupport { def gteqSpark3_2_1: Boolean = getSparkVersion >= "3.2.1" def gteqSpark3_2_2: Boolean = getSparkVersion >= "3.2.2" def gteqSpark3_3: Boolean = getSparkVersion >= "3.3" + def gteqSpark3_3_2: Boolean = getSparkVersion >= "3.3.2" } -object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport { +object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport with Logging { override def getSparkVersion: String = SPARK_VERSION @@ -120,6 +127,80 @@ object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport { sparkAdapter.createSparkRowSerDe(structType) } + def parsePartitionColumnValues(partitionColumns: Array[String], + partitionPath: String, + basePath: Path, + schema: StructType, + timeZoneId: String, + sparkParsePartitionUtil: SparkParsePartitionUtil, + shouldValidatePartitionCols: Boolean): Array[Object] = { + if (partitionColumns.length == 0) { + // This is a non-partitioned table + Array.empty + } else { + val partitionFragments = partitionPath.split("/") + if (partitionFragments.length != partitionColumns.length) { + if (partitionColumns.length == 1) { + // If the partition column size is not equal to the partition fragment size + // and the partition column size is 1, we map the whole partition path + // to the partition column which can benefit from the partition prune. + val prefix = s"${partitionColumns.head}=" + val partitionValue = if (partitionPath.startsWith(prefix)) { + // support hive style partition path + partitionPath.substring(prefix.length) + } else { + partitionPath + } + Array(UTF8String.fromString(partitionValue)) + } else { + // If the partition column size is not equal to the partition fragments size + // and the partition column size > 1, we do not know how to map the partition + // fragments to the partition columns and therefore return an empty tuple. We don't + // fail outright so that in some cases we can fallback to reading the table as non-partitioned + // one + logWarning(s"Failed to parse partition values: found partition fragments" + + s" (${partitionFragments.mkString(",")}) are not aligned with expected partition columns" + + s" (${partitionColumns.mkString(",")})") + Array.empty + } + } else { + // If partitionSeqs.length == partitionSchema.fields.length + // Append partition name to the partition value if the + // HIVE_STYLE_PARTITIONING is disable. + // e.g. convert "/xx/xx/2021/02" to "/xx/xx/year=2021/month=02" + val partitionWithName = + partitionFragments.zip(partitionColumns).map { + case (partition, columnName) => + if (partition.indexOf("=") == -1) { + s"${columnName}=$partition" + } else { + partition + } + }.mkString("/") + + val pathWithPartitionName = new CachingPath(basePath, CachingPath.createRelativePathUnsafe(partitionWithName)) + val partitionSchema = StructType(schema.fields.filter(f => partitionColumns.contains(f.name))) + val partitionValues = parsePartitionPath(pathWithPartitionName, partitionSchema, timeZoneId, + sparkParsePartitionUtil, basePath, shouldValidatePartitionCols) + partitionValues.map(_.asInstanceOf[Object]).toArray + } + } + } + + private def parsePartitionPath(partitionPath: Path, partitionSchema: StructType, timeZoneId: String, + sparkParsePartitionUtil: SparkParsePartitionUtil, basePath: Path, + shouldValidatePartitionCols: Boolean): Seq[Any] = { + val partitionDataTypes = partitionSchema.map(f => f.name -> f.dataType).toMap + sparkParsePartitionUtil.parsePartition( + partitionPath, + typeInference = false, + Set(basePath), + partitionDataTypes, + getTimeZone(timeZoneId), + validatePartitionValues = shouldValidatePartitionCols + ).toSeq(partitionSchema) + } + private def injectSQLConf[T: ClassTag](rdd: RDD[T], conf: SQLConf): RDD[T] = new SQLConfInjectingRDD(rdd, conf) } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala index 4cdbbf7577abd..a767fd3c5bfef 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/SparkKeyGenUtils.scala @@ -22,7 +22,6 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.{BaseKeyGenerator, CustomAvroKeyGenerator, CustomKeyGenerator, KeyGenerator} import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory -import java.util.Properties import scala.collection.JavaConverters._ object SparkKeyGenUtils { @@ -31,8 +30,7 @@ object SparkKeyGenUtils { * @param properties config properties * @return partition columns */ - def getPartitionColumns(properties: Properties): String = { - val props = new TypedProperties(properties) + def getPartitionColumns(props: TypedProperties): String = { val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props) getPartitionColumns(keyGenerator, props) } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala index 3894065d8090b..9d7fa3b784fc4 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala @@ -18,11 +18,12 @@ package org.apache.spark -import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.io.{Input, Output} +import com.esotericsoftware.kryo.{Kryo, Serializer} import com.esotericsoftware.kryo.serializers.JavaSerializer import org.apache.hudi.client.model.HoodieInternalRow import org.apache.hudi.common.config.SerializableConfiguration -import org.apache.hudi.common.model.HoodieSparkRecord +import org.apache.hudi.common.model.{HoodieKey, HoodieSparkRecord} import org.apache.hudi.common.util.HoodieCommonKryoRegistrar import org.apache.hudi.config.HoodieWriteConfig import org.apache.spark.serializer.KryoRegistrator @@ -44,12 +45,15 @@ import org.apache.spark.serializer.KryoRegistrator * */ class HoodieSparkKryoRegistrar extends HoodieCommonKryoRegistrar with KryoRegistrator { + override def registerClasses(kryo: Kryo): Unit = { /////////////////////////////////////////////////////////////////////////// // NOTE: DO NOT REORDER REGISTRATIONS /////////////////////////////////////////////////////////////////////////// super[HoodieCommonKryoRegistrar].registerClasses(kryo) + kryo.register(classOf[HoodieKey], new HoodieKeySerializer) + kryo.register(classOf[HoodieWriteConfig]) kryo.register(classOf[HoodieSparkRecord]) @@ -59,6 +63,23 @@ class HoodieSparkKryoRegistrar extends HoodieCommonKryoRegistrar with KryoRegist // we're relying on [[SerializableConfiguration]] wrapper to work it around kryo.register(classOf[SerializableConfiguration], new JavaSerializer()) } + + /** + * NOTE: This {@link Serializer} could deserialize instance of {@link HoodieKey} serialized + * by implicitly generated Kryo serializer (based on {@link com.esotericsoftware.kryo.serializers.FieldSerializer} + */ + class HoodieKeySerializer extends Serializer[HoodieKey] { + override def write(kryo: Kryo, output: Output, key: HoodieKey): Unit = { + output.writeString(key.getRecordKey) + output.writeString(key.getPartitionPath) + } + + override def read(kryo: Kryo, input: Input, klass: Class[HoodieKey]): HoodieKey = { + val recordKey = input.readString() + val partitionPath = input.readString() + new HoodieKey(recordKey, partitionPath) + } + } } object HoodieSparkKryoRegistrar { diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala index dfe3295cf003c..b56b0b1e4ce64 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieInternalRowUtils.scala @@ -18,304 +18,416 @@ package org.apache.spark.sql -import java.nio.charset.StandardCharsets -import java.util.HashMap -import java.util.concurrent.ConcurrentHashMap import org.apache.avro.Schema import org.apache.hbase.thirdparty.com.google.common.base.Supplier -import org.apache.hudi.AvroConversionUtils +import org.apache.hudi.AvroConversionUtils.convertAvroSchemaToStructType import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, toJavaDate} import org.apache.hudi.exception.HoodieException -import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +import org.apache.spark.sql.HoodieCatalystExpressionUtils.generateUnsafeProjection +import org.apache.spark.sql.HoodieUnsafeRowUtils.{NestedFieldPath, composeNestedFieldPath} +import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.types.Decimal.ROUND_HALF_UP import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String + +import java.util.concurrent.ConcurrentHashMap +import java.util.{ArrayDeque => JArrayDeque, Collections => JCollections, Deque => JDeque, Map => JMap} +import java.util.function.{Function => JFunction} import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter object HoodieInternalRowUtils { - // Projection are all thread local. Projection is not thread-safe - val unsafeProjectionThreadLocal: ThreadLocal[HashMap[(StructType, StructType), UnsafeProjection]] = - ThreadLocal.withInitial(new Supplier[HashMap[(StructType, StructType), UnsafeProjection]] { - override def get(): HashMap[(StructType, StructType), UnsafeProjection] = new HashMap[(StructType, StructType), UnsafeProjection] + private type RenamedColumnMap = JMap[String, String] + private type UnsafeRowWriter = InternalRow => UnsafeRow + + // NOTE: [[UnsafeProjection]] objects cache have to stay [[ThreadLocal]] since these are not thread-safe + private val unsafeWriterThreadLocal: ThreadLocal[mutable.HashMap[(StructType, StructType, RenamedColumnMap), UnsafeRowWriter]] = + ThreadLocal.withInitial(new Supplier[mutable.HashMap[(StructType, StructType, RenamedColumnMap), UnsafeRowWriter]] { + override def get(): mutable.HashMap[(StructType, StructType, RenamedColumnMap), UnsafeRowWriter] = + new mutable.HashMap[(StructType, StructType, RenamedColumnMap), UnsafeRowWriter] }) - val schemaMap = new ConcurrentHashMap[Schema, StructType] - val orderPosListMap = new ConcurrentHashMap[(StructType, String), NestedFieldPath] - /** - * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecord(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema) - */ - def rewriteRecord(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType): InternalRow = { - val newRow = new GenericInternalRow(Array.fill(newSchema.fields.length)(null).asInstanceOf[Array[Any]]) - - for ((field, pos) <- newSchema.fields.zipWithIndex) { - var oldValue: AnyRef = null - var oldType: DataType = null - if (existField(oldSchema, field.name)) { - val oldField = oldSchema(field.name) - val oldPos = oldSchema.fieldIndex(field.name) - oldType = oldField.dataType - oldValue = oldRecord.get(oldPos, oldType) - } - if (oldValue != null) { - field.dataType match { - case structType: StructType => - val oldType = oldSchema(field.name).dataType.asInstanceOf[StructType] - val newValue = rewriteRecord(oldValue.asInstanceOf[InternalRow], oldType, structType) - newRow.update(pos, newValue) - case decimalType: DecimalType => - val oldFieldSchema = oldSchema(field.name).dataType.asInstanceOf[DecimalType] - if (decimalType.scale != oldFieldSchema.scale || decimalType.precision != oldFieldSchema.precision) { - newRow.update(pos, Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale)) - ) - } else { - newRow.update(pos, oldValue) - } - case t if t == oldType => newRow.update(pos, oldValue) - // Type promotion - case _: ShortType => - oldType match { - case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toShort) - case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") - } - case _: IntegerType => - oldType match { - case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toInt) - case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toInt) - case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") - } - case _: LongType => - oldType match { - case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toLong) - case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toLong) - case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toLong) - case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") - } - case _: FloatType => - oldType match { - case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toFloat) - case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toFloat) - case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toFloat) - case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toFloat) - case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") - } - case _: DoubleType => - oldType match { - case _: ByteType => newRow.update(pos, oldValue.asInstanceOf[Byte].toDouble) - case _: ShortType => newRow.update(pos, oldValue.asInstanceOf[Short].toDouble) - case _: IntegerType => newRow.update(pos, oldValue.asInstanceOf[Int].toDouble) - case _: LongType => newRow.update(pos, oldValue.asInstanceOf[Long].toDouble) - case _: FloatType => newRow.update(pos, oldValue.asInstanceOf[Float].toDouble) - case _ => throw new IllegalArgumentException(s"$oldSchema and $newSchema are incompatible") - } - case _: BinaryType if oldType.isInstanceOf[StringType] => newRow.update(pos, oldValue.asInstanceOf[String].getBytes) - case _ => newRow.update(pos, oldValue) - } - } else { - // TODO default value in newSchema - } - } + // NOTE: [[UnsafeRowWriter]] objects cache have to stay [[ThreadLocal]] since these are not thread-safe + private val unsafeProjectionThreadLocal: ThreadLocal[mutable.HashMap[(StructType, StructType), UnsafeProjection]] = + ThreadLocal.withInitial(new Supplier[mutable.HashMap[(StructType, StructType), UnsafeProjection]] { + override def get(): mutable.HashMap[(StructType, StructType), UnsafeProjection] = + new mutable.HashMap[(StructType, StructType), UnsafeProjection] + }) - newRow - } + private val schemaMap = new ConcurrentHashMap[Schema, StructType] + private val orderPosListMap = new ConcurrentHashMap[(StructType, String), Option[NestedFieldPath]] /** - * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(org.apache.avro.generic.IndexedRecord, org.apache.avro.Schema, java.util.Map) + * Provides cached instance of [[UnsafeProjection]] transforming provided [[InternalRow]]s from + * one [[StructType]] and into another [[StructType]] + * + * For more details regarding its semantic, please check corresponding scala-doc for + * [[HoodieCatalystExpressionUtils.generateUnsafeProjection]] */ - def rewriteRecordWithNewSchema(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType, renameCols: java.util.Map[String, String]): InternalRow = { - rewriteRecordWithNewSchema(oldRecord, oldSchema, newSchema, renameCols, new java.util.LinkedList[String]).asInstanceOf[InternalRow] + def getCachedUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = { + unsafeProjectionThreadLocal.get() + .getOrElseUpdate((from, to), generateUnsafeProjection(from, to)) } /** - * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(java.lang.Object, org.apache.avro.Schema, org.apache.avro.Schema, java.util.Map, java.util.Deque) + * Provides cached instance of [[UnsafeRowWriter]] transforming provided [[InternalRow]]s from + * one [[StructType]] and into another [[StructType]] + * + * Unlike [[UnsafeProjection]] requiring that [[from]] has to be a proper subset of [[to]] schema, + * [[UnsafeRowWriter]] is able to perform whole spectrum of schema-evolution transformations including: + * + *

    + *
  • Transforming nested structs/maps/arrays
  • + *
  • Handling type promotions (int -> long, etc)
  • + *
  • Handling (field) renames
  • + *
*/ - private def rewriteRecordWithNewSchema(oldRecord: Any, oldSchema: DataType, newSchema: DataType, renameCols: java.util.Map[String, String], fieldNames: java.util.Deque[String]): Any = { - if (oldRecord == null) { - null - } else { - newSchema match { - case targetSchema: StructType => - if (!oldRecord.isInstanceOf[InternalRow]) { - throw new IllegalArgumentException("cannot rewrite record with different type") - } - val oldRow = oldRecord.asInstanceOf[InternalRow] - val helper = mutable.Map[Integer, Any]() - - val oldStrucType = oldSchema.asInstanceOf[StructType] - targetSchema.fields.zipWithIndex.foreach { case (field, i) => - fieldNames.push(field.name) - if (existField(oldStrucType, field.name)) { - val oldField = oldStrucType(field.name) - val oldPos = oldStrucType.fieldIndex(field.name) - helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames) - } else { - val fieldFullName = createFullName(fieldNames) - val colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.") - val lastColNameFromOldSchema = colNamePartsFromOldSchema(colNamePartsFromOldSchema.length - 1) - // deal with rename - if (!existField(oldStrucType, field.name) && existField(oldStrucType, lastColNameFromOldSchema)) { - // find rename - val oldField = oldStrucType(lastColNameFromOldSchema) - val oldPos = oldStrucType.fieldIndex(lastColNameFromOldSchema) - helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames) - } - } - fieldNames.pop() - } - val newRow = new GenericInternalRow(Array.fill(targetSchema.length)(null).asInstanceOf[Array[Any]]) - targetSchema.fields.zipWithIndex.foreach { case (_, i) => - if (helper.contains(i)) { - newRow.update(i, helper(i)) - } else { - // TODO add default val - newRow.update(i, null) - } - } + def getCachedUnsafeRowWriter(from: StructType, to: StructType, renamedColumnsMap: JMap[String, String] = JCollections.emptyMap()): UnsafeRowWriter = { + unsafeWriterThreadLocal.get() + .getOrElseUpdate((from, to, renamedColumnsMap), genUnsafeRowWriter(from, to, renamedColumnsMap)) + } - newRow - case targetSchema: ArrayType => - if (!oldRecord.isInstanceOf[ArrayData]) { - throw new IllegalArgumentException("cannot rewrite record with different type") - } - val oldElementType = oldSchema.asInstanceOf[ArrayType].elementType - val oldArray = oldRecord.asInstanceOf[ArrayData] - val newElementType = targetSchema.elementType - val newArray = new GenericArrayData(Array.fill(oldArray.numElements())(null).asInstanceOf[Array[Any]]) - fieldNames.push("element") - oldArray.toSeq[Any](oldElementType).zipWithIndex.foreach { case (value, i) => newArray.update(i, rewriteRecordWithNewSchema(value.asInstanceOf[AnyRef], oldElementType, newElementType, renameCols, fieldNames)) } - fieldNames.pop() - - newArray - case targetSchema: MapType => - if (!oldRecord.isInstanceOf[MapData]) { - throw new IllegalArgumentException("cannot rewrite record with different type") - } - val oldValueType = oldSchema.asInstanceOf[MapType].valueType - val oldKeyType = oldSchema.asInstanceOf[MapType].keyType - val oldMap = oldRecord.asInstanceOf[MapData] - val newValueType = targetSchema.valueType - val newKeyArray = new GenericArrayData(Array.fill(oldMap.keyArray().numElements())(null).asInstanceOf[Array[Any]]) - val newValueArray = new GenericArrayData(Array.fill(oldMap.valueArray().numElements())(null).asInstanceOf[Array[Any]]) - val newMap = new ArrayBasedMapData(newKeyArray, newValueArray) - fieldNames.push("value") - oldMap.keyArray().toSeq[Any](oldKeyType).zipWithIndex.foreach { case (value, i) => newKeyArray.update(i, rewritePrimaryType(value, oldKeyType, oldKeyType)) } - oldMap.valueArray().toSeq[Any](oldValueType).zipWithIndex.foreach { case (value, i) => newValueArray.update(i, rewriteRecordWithNewSchema(value.asInstanceOf[AnyRef], oldValueType, newValueType, renameCols, fieldNames)) } - fieldNames.pop() - - newMap - case _ => rewritePrimaryType(oldRecord, oldSchema, newSchema) - } + def getCachedPosList(structType: StructType, field: String): Option[NestedFieldPath] = { + val nestedFieldPathOpt = orderPosListMap.get((structType, field)) + // NOTE: This specifically designed to do 2 lookups (in case of cache-miss) to avoid + // allocating the closure when using [[computeIfAbsent]] on more frequent cache-hit path + if (nestedFieldPathOpt != null) { + nestedFieldPathOpt + } else { + orderPosListMap.computeIfAbsent((structType, field), new JFunction[(StructType, String), Option[NestedFieldPath]] { + override def apply(t: (StructType, String)): Option[NestedFieldPath] = + composeNestedFieldPath(structType, field) + }) } } - def getCachedPosList(structType: StructType, field: String): NestedFieldPath = { - val schemaPair = (structType, field) - if (!orderPosListMap.containsKey(schemaPair)) { - val posList = HoodieUnsafeRowUtils.composeNestedFieldPath(structType, field) - orderPosListMap.put(schemaPair, posList) + def getCachedSchema(schema: Schema): StructType = { + val structType = schemaMap.get(schema) + // NOTE: This specifically designed to do 2 lookups (in case of cache-miss) to avoid + // allocating the closure when using [[computeIfAbsent]] on more frequent cache-hit path + if (structType != null) { + structType + } else { + schemaMap.computeIfAbsent(schema, new JFunction[Schema, StructType] { + override def apply(t: Schema): StructType = + convertAvroSchemaToStructType(schema) + }) } - orderPosListMap.get(schemaPair) } - def getCachedUnsafeProjection(from: StructType, to: StructType): UnsafeProjection = { - val schemaPair = (from, to) - val map = unsafeProjectionThreadLocal.get() - if (!map.containsKey(schemaPair)) { - val projection = HoodieCatalystExpressionUtils.generateUnsafeProjection(from, to) - map.put(schemaPair, projection) + private[sql] def genUnsafeRowWriter(prevSchema: StructType, + newSchema: StructType, + renamedColumnsMap: JMap[String, String]): UnsafeRowWriter = { + val writer = newWriterRenaming(prevSchema, newSchema, renamedColumnsMap, new JArrayDeque[String]()) + val unsafeProjection = generateUnsafeProjection(newSchema, newSchema) + val phonyUpdater = new CatalystDataUpdater { + var value: InternalRow = _ + + override def set(ordinal: Int, value: Any): Unit = + this.value = value.asInstanceOf[InternalRow] } - map.get(schemaPair) - } - def getCachedSchema(schema: Schema): StructType = { - if (!schemaMap.containsKey(schema)) { - val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) - schemaMap.put(schema, structType) + oldRow => { + writer(phonyUpdater, 0, oldRow) + unsafeProjection(phonyUpdater.value) } - schemaMap.get(schema) } - def existField(structType: StructType, name: String): Boolean = { - try { - HoodieUnsafeRowUtils.composeNestedFieldPath(structType, name) - true - } catch { - case _: IllegalArgumentException => false + private type RowFieldUpdater = (CatalystDataUpdater, Int, Any) => Unit + + private def genUnsafeStructWriter(prevStructType: StructType, + newStructType: StructType, + renamedColumnsMap: JMap[String, String], + fieldNamesStack: JDeque[String]): (CatalystDataUpdater, Any) => Unit = { + // TODO need to canonicalize schemas (casing) + val fieldWriters = ArrayBuffer.empty[RowFieldUpdater] + val positionMap = ArrayBuffer.empty[Int] + + for (newField <- newStructType.fields) { + fieldNamesStack.push(newField.name) + + val (fieldWriter, prevFieldPos): (RowFieldUpdater, Int) = + prevStructType.getFieldIndex(newField.name) match { + case Some(prevFieldPos) => + val prevField = prevStructType(prevFieldPos) + (newWriterRenaming(prevField.dataType, newField.dataType, renamedColumnsMap, fieldNamesStack), prevFieldPos) + + case None => + val newFieldQualifiedName = createFullName(fieldNamesStack) + val prevFieldName: String = lookupRenamedField(newFieldQualifiedName, renamedColumnsMap) + + // Handle rename + prevStructType.getFieldIndex(prevFieldName) match { + case Some(prevFieldPos) => + val prevField = prevStructType.fields(prevFieldPos) + (newWriterRenaming(prevField.dataType, newField.dataType, renamedColumnsMap, fieldNamesStack), prevFieldPos) + + case None => + val updater: RowFieldUpdater = (fieldUpdater, ordinal, _) => fieldUpdater.setNullAt(ordinal) + (updater, -1) + } + } + + fieldWriters += fieldWriter + positionMap += prevFieldPos + + fieldNamesStack.pop() } - } - private def rewritePrimaryType(oldValue: Any, oldSchema: DataType, newSchema: DataType) = { - if (oldSchema.equals(newSchema) || (oldSchema.isInstanceOf[DecimalType] && newSchema.isInstanceOf[DecimalType])) { - oldSchema match { - case NullType | BooleanType | IntegerType | LongType | FloatType | DoubleType | DateType | TimestampType | BinaryType => - oldValue - // Copy UTF8String before putting into GenericInternalRow - case StringType => UTF8String.fromString(oldValue.toString) - case DecimalType() => - Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale)) - case _ => - throw new HoodieException("Unknown schema type: " + newSchema) + (fieldUpdater, row) => { + var pos = 0 + while (pos < fieldWriters.length) { + val prevPos = positionMap(pos) + val prevValue = if (prevPos >= 0) { + row.asInstanceOf[InternalRow].get(prevPos, prevStructType.fields(prevPos).dataType) + } else { + null + } + + if(prevValue == null) + fieldUpdater.setNullAt(pos) + else + fieldWriters(pos)(fieldUpdater, pos, prevValue) + pos += 1 } - } else { - rewritePrimaryTypeWithDiffSchemaType(oldValue, oldSchema, newSchema) } } - private def rewritePrimaryTypeWithDiffSchemaType(oldValue: Any, oldSchema: DataType, newSchema: DataType): Any = { - val value = newSchema match { - case NullType | BooleanType => - case DateType if oldSchema.equals(StringType) => - CatalystTypeConverters.convertToCatalyst(java.sql.Date.valueOf(oldValue.toString)) - case LongType => - oldSchema match { - case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].longValue()) + private def newWriterRenaming(prevDataType: DataType, + newDataType: DataType, + renamedColumnsMap: JMap[String, String], + fieldNameStack: JDeque[String]): RowFieldUpdater = { + (newDataType, prevDataType) match { + case (newType, prevType) if prevType == newType => + (fieldUpdater, ordinal, value) => fieldUpdater.set(ordinal, value) + + case (newStructType: StructType, prevStructType: StructType) => + val writer = genUnsafeStructWriter(prevStructType, newStructType, renamedColumnsMap, fieldNameStack) + + val newRow = new SpecificInternalRow(newStructType.fields.map(_.dataType)) + val rowUpdater = new RowUpdater(newRow) + + (fieldUpdater, ordinal, value) => { + // Here new row is built in 2 stages: + // - First, we pass mutable row (used as buffer/scratchpad) created above wrapped into [[RowUpdater]] + // into generated row-writer + // - Upon returning from row-writer, we call back into parent row's [[fieldUpdater]] to set returned + // row as a value in it + writer(rowUpdater, value) + fieldUpdater.set(ordinal, newRow) + } + + case (ArrayType(newElementType, _), ArrayType(prevElementType, containsNull)) => + fieldNameStack.push("element") + val elementWriter = newWriterRenaming(prevElementType, newElementType, renamedColumnsMap, fieldNameStack) + fieldNameStack.pop() + + (fieldUpdater, ordinal, value) => { + val prevArrayData = value.asInstanceOf[ArrayData] + val prevArray = prevArrayData.toObjectArray(prevElementType) + + val newArrayData = createArrayData(newElementType, prevArrayData.numElements()) + val elementUpdater = new ArrayDataUpdater(newArrayData) + + var i = 0 + while (i < prevArray.length) { + val element = prevArray(i) + if (element == null) { + if (!containsNull) { + throw new HoodieException( + s"Array value at path '${fieldNameStack.asScala.mkString(".")}' is not allowed to be null") + } else { + elementUpdater.setNullAt(i) + } + } else { + elementWriter(elementUpdater, i, element) + } + i += 1 + } + + fieldUpdater.set(ordinal, newArrayData) + } + + case (MapType(_, newValueType, _), MapType(_, prevValueType, valueContainsNull)) => + fieldNameStack.push("value") + val valueWriter = newWriterRenaming(prevValueType, newValueType, renamedColumnsMap, fieldNameStack) + fieldNameStack.pop() + + (updater, ordinal, value) => + val mapData = value.asInstanceOf[MapData] + val prevKeyArrayData = mapData.keyArray + val prevValueArrayData = mapData.valueArray + val prevValueArray = prevValueArrayData.toObjectArray(prevValueType) + + val newValueArray = createArrayData(newValueType, mapData.numElements()) + val valueUpdater = new ArrayDataUpdater(newValueArray) + var i = 0 + while (i < prevValueArray.length) { + val value = prevValueArray(i) + if (value == null) { + if (!valueContainsNull) { + throw new HoodieException(s"Map value at path ${fieldNameStack.asScala.mkString(".")} is not allowed to be null") + } else { + valueUpdater.setNullAt(i) + } + } else { + valueWriter(valueUpdater, i, value) + } + i += 1 + } + + // NOTE: Key's couldn't be transformed and have to always be of [[StringType]] + updater.set(ordinal, new ArrayBasedMapData(prevKeyArrayData, newValueArray)) + + case (newDecimal: DecimalType, _) => + prevDataType match { + case IntegerType | LongType | FloatType | DoubleType | StringType => + (fieldUpdater, ordinal, value) => + val scale = newDecimal.scale + // TODO this has to be revisited to avoid loss of precision (for fps) + fieldUpdater.setDecimal(ordinal, Decimal.fromDecimal(BigDecimal(value.toString).setScale(scale, ROUND_HALF_UP))) + + case _: DecimalType => + (fieldUpdater, ordinal, value) => + fieldUpdater.setDecimal(ordinal, Decimal.fromDecimal(value.asInstanceOf[Decimal].toBigDecimal.setScale(newDecimal.scale))) + case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case FloatType => - oldSchema match { - case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].floatValue()) - case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].floatValue()) + + case (_: ShortType, _) => + prevDataType match { + case _: ByteType => (fieldUpdater, ordinal, value) => fieldUpdater.setShort(ordinal, value.asInstanceOf[Byte].toShort) case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case DoubleType => - oldSchema match { - case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].doubleValue()) - case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].doubleValue()) - case FloatType => CatalystTypeConverters.convertToCatalyst(java.lang.Double.valueOf(oldValue.asInstanceOf[Float] + "")) + + case (_: IntegerType, _) => + prevDataType match { + case _: ShortType => (fieldUpdater, ordinal, value) => fieldUpdater.setInt(ordinal, value.asInstanceOf[Short].toInt) + case _: ByteType => (fieldUpdater, ordinal, value) => fieldUpdater.setInt(ordinal, value.asInstanceOf[Byte].toInt) case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case BinaryType => - oldSchema match { - case StringType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[String].getBytes(StandardCharsets.UTF_8)) + + case (_: LongType, _) => + prevDataType match { + case _: IntegerType => (fieldUpdater, ordinal, value) => fieldUpdater.setLong(ordinal, value.asInstanceOf[Int].toLong) + case _: ShortType => (fieldUpdater, ordinal, value) => fieldUpdater.setLong(ordinal, value.asInstanceOf[Short].toLong) + case _: ByteType => (fieldUpdater, ordinal, value) => fieldUpdater.setLong(ordinal, value.asInstanceOf[Byte].toLong) case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case StringType => - oldSchema match { - case BinaryType => CatalystTypeConverters.convertToCatalyst(new String(oldValue.asInstanceOf[Array[Byte]])) - case DateType => CatalystTypeConverters.convertToCatalyst(toJavaDate(oldValue.asInstanceOf[Integer]).toString) - case IntegerType | LongType | FloatType | DoubleType | DecimalType() => CatalystTypeConverters.convertToCatalyst(oldValue.toString) + + case (_: FloatType, _) => + prevDataType match { + case _: LongType => (fieldUpdater, ordinal, value) => fieldUpdater.setFloat(ordinal, value.asInstanceOf[Long].toFloat) + case _: IntegerType => (fieldUpdater, ordinal, value) => fieldUpdater.setFloat(ordinal, value.asInstanceOf[Int].toFloat) + case _: ShortType => (fieldUpdater, ordinal, value) => fieldUpdater.setFloat(ordinal, value.asInstanceOf[Short].toFloat) + case _: ByteType => (fieldUpdater, ordinal, value) => fieldUpdater.setFloat(ordinal, value.asInstanceOf[Byte].toFloat) case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case DecimalType() => - oldSchema match { - case IntegerType | LongType | FloatType | DoubleType | StringType => - val scale = newSchema.asInstanceOf[DecimalType].scale - Decimal.fromDecimal(BigDecimal(oldValue.toString).setScale(scale)) + case (_: DoubleType, _) => + prevDataType match { + case _: FloatType => (fieldUpdater, ordinal, value) => fieldUpdater.setDouble(ordinal, value.asInstanceOf[Float].toDouble) + case _: LongType => (fieldUpdater, ordinal, value) => fieldUpdater.setDouble(ordinal, value.asInstanceOf[Long].toDouble) + case _: IntegerType => (fieldUpdater, ordinal, value) => fieldUpdater.setDouble(ordinal, value.asInstanceOf[Int].toDouble) + case _: ShortType => (fieldUpdater, ordinal, value) => fieldUpdater.setDouble(ordinal, value.asInstanceOf[Short].toDouble) + case _: ByteType => (fieldUpdater, ordinal, value) => fieldUpdater.setDouble(ordinal, value.asInstanceOf[Byte].toDouble) case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } - case _ => - } - if (value == None) { - throw new HoodieException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema)) - } else { - CatalystTypeConverters.convertToCatalyst(value) + + case (_: BinaryType, _: StringType) => + (fieldUpdater, ordinal, value) => fieldUpdater.set(ordinal, value.asInstanceOf[UTF8String].getBytes) + + // TODO revisit this (we need to align permitted casting w/ Spark) + // NOTE: This is supported to stay compatible w/ [[HoodieAvroUtils.rewriteRecordWithNewSchema]] + case (_: StringType, _) => + prevDataType match { + case BinaryType => (fieldUpdater, ordinal, value) => + fieldUpdater.set(ordinal, UTF8String.fromBytes(value.asInstanceOf[Array[Byte]])) + case DateType => (fieldUpdater, ordinal, value) => + fieldUpdater.set(ordinal, UTF8String.fromString(toJavaDate(value.asInstanceOf[Integer]).toString)) + case IntegerType | LongType | FloatType | DoubleType | _: DecimalType => + (fieldUpdater, ordinal, value) => fieldUpdater.set(ordinal, UTF8String.fromString(value.toString)) + + case _ => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") + } + + case (DateType, StringType) => + (fieldUpdater, ordinal, value) => + fieldUpdater.set(ordinal, CatalystTypeConverters.convertToCatalyst(java.sql.Date.valueOf(value.toString))) + + case (_, _) => + throw new IllegalArgumentException(s"$prevDataType and $newDataType are incompatible") } } - def removeFields(schema: StructType, fieldsToRemove: java.util.List[String]): StructType = { - StructType(schema.fields.filter(field => !fieldsToRemove.contains(field.name))) + private def lookupRenamedField(newFieldQualifiedName: String, renamedColumnsMap: JMap[String, String]) = { + val prevFieldQualifiedName = renamedColumnsMap.getOrDefault(newFieldQualifiedName, "") + val prevFieldQualifiedNameParts = prevFieldQualifiedName.split("\\.") + val prevFieldName = prevFieldQualifiedNameParts(prevFieldQualifiedNameParts.length - 1) + + prevFieldName } + + private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match { + case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length)) + case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length)) + case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length)) + case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length)) + case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length)) + case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length)) + case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length)) + case _ => new GenericArrayData(new Array[Any](length)) + } + + sealed trait CatalystDataUpdater { + def set(ordinal: Int, value: Any): Unit + def setNullAt(ordinal: Int): Unit = set(ordinal, null) + def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value) + def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value) + def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value) + def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value) + def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value) + def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value) + def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value) + def setDecimal(ordinal: Int, value: Decimal): Unit = set(ordinal, value) + } + + final class RowUpdater(row: InternalRow) extends CatalystDataUpdater { + override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value) + override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal) + override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value) + override def setDecimal(ordinal: Int, value: Decimal): Unit = + row.setDecimal(ordinal, value, value.precision) + } + + final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater { + override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value) + override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal) + override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value) + override def setDecimal(ordinal: Int, value: Decimal): Unit = array.update(ordinal, value) + } + } diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala index c105142de0f45..5486c9f6551a5 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeRowUtils.scala @@ -93,29 +93,36 @@ object HoodieUnsafeRowUtils { * * This method produces nested-field path, that is subsequently used by [[getNestedInternalRowValue]], [[getNestedRowValue]] */ - def composeNestedFieldPath(schema: StructType, nestedFieldRef: String): NestedFieldPath = { + def composeNestedFieldPath(schema: StructType, nestedFieldRef: String): Option[NestedFieldPath]= { val fieldRefParts = nestedFieldRef.split('.') val ordSeq = ArrayBuffer[(Int, StructField)]() var curSchema = schema var idx = 0 while (idx < fieldRefParts.length) { val fieldRefPart = fieldRefParts(idx) - val ord = curSchema.fieldIndex(fieldRefPart) - val field = curSchema(ord) - // Append current field's (ordinal, data-type) - ordSeq.append((ord, field)) - // Update current schema, unless terminal field-ref part - if (idx < fieldRefParts.length - 1) { - curSchema = field.dataType match { - case st: StructType => st - case dt@_ => - throw new IllegalArgumentException(s"Invalid nested field reference ${fieldRefParts.drop(idx).mkString(".")} into $dt") - } + curSchema.getFieldIndex(fieldRefPart) match { + case Some(ord) => + val field = curSchema(ord) + // Append current field's (ordinal, data-type) + ordSeq.append((ord, field)) + // Update current schema, unless terminal field-ref part + if (idx < fieldRefParts.length - 1) { + curSchema = field.dataType match { + case st: StructType => st + case _ => + // In case we've stumbled upon something other than the [[StructType]] means that + // provided nested field reference is invalid. In that case we simply return null + return None + } + } + + // In case, field is not found we return null + case None => return None } idx += 1 } - NestedFieldPath(ordSeq.toArray) + Some(NestedFieldPath(ordSeq.toArray)) } case class NestedFieldPath(parts: Array[(Int, StructField)]) diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala index dfd416b6f5280..ee22f714c9c90 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieUnsafeUtils.scala @@ -22,7 +22,8 @@ import org.apache.hudi.HoodieUnsafeRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair @@ -38,8 +39,21 @@ object HoodieUnsafeUtils { * but instead will just execute Spark resolution, optimization and actual execution planning stages * returning instance of [[SparkPlan]] ready for execution */ - def getOutputPartitioning(df: DataFrame): Partitioning = - df.queryExecution.executedPlan.outputPartitioning + def getNumPartitions(df: DataFrame): Int = { + // NOTE: In general we'd rely on [[outputPartitioning]] of the executable [[SparkPlan]] to determine + // number of partitions plan is going to be executed with. + // However in case of [[LogicalRDD]] plan's output-partitioning will be stubbed as [[UnknownPartitioning]] + // and therefore we will be falling back to determine number of partitions by looking at the RDD itself + df.queryExecution.logical match { + case LogicalRDD(_, rdd, outputPartitioning, _, _) => + outputPartitioning match { + case _: UnknownPartitioning => rdd.getNumPartitions + case _ => outputPartitioning.numPartitions + } + + case _ => df.queryExecution.executedPlan.outputPartitioning.numPartitions + } + } /** * Creates [[DataFrame]] from provided [[plan]] diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala index f4edf7fb94260..67c68fd0fab1b 100644 --- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala +++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala @@ -23,20 +23,21 @@ import org.apache.hadoop.fs.Path import org.apache.hudi.client.utils.SparkRowSerDe import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.util.TablePathUtils +import org.apache.spark.sql._ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConverters, HoodieAvroSerializer} -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.util.DateFormatter +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql._ import org.apache.spark.storage.StorageLevel -import java.util.Locale +import java.util.{Locale, TimeZone} /** * Interface adapting discrepancies and incompatibilities between different Spark versions @@ -108,6 +109,11 @@ trait SparkAdapter extends Serializable { */ def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String] + /** + * Get the [[DateFormatter]]. + */ + def getDateFormatter(tz: TimeZone): DateFormatter + /** * Combine [[PartitionedFile]] to [[FilePartition]] according to `maxSplitBytes`. */ diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java new file mode 100644 index 0000000000000..aeda164e7d06a --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSparkRDDWriteClient.java @@ -0,0 +1,81 @@ +/* + * 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.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.io.IOException; +import java.net.URI; +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class TestSparkRDDWriteClient extends SparkClientFunctionalTestHarness { + + @ParameterizedTest + @CsvSource({"true,true", "true,false", "false,true", "false,false"}) + public void testWriteClientAndTableServiceClientWithTimelineServer( + boolean enableEmbeddedTimelineServer, boolean passInTimelineServer) throws IOException { + HoodieTableMetaClient metaClient = + getHoodieMetaClient(hadoopConf(), URI.create(basePath()).getPath(), new Properties()); + HoodieWriteConfig writeConfig = getConfigBuilder(true) + .withPath(metaClient.getBasePathV2().toString()) + .withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineServer) + .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() + .withRemoteServerPort(incrementTimelineServicePortToUse()).build()) + .build(); + + SparkRDDWriteClient writeClient; + if (passInTimelineServer) { + EmbeddedTimelineService timelineService = + new EmbeddedTimelineService(context(), null, writeConfig); + timelineService.startServer(); + writeConfig.setViewStorageConfig(timelineService.getRemoteFileSystemViewConfig()); + writeClient = new SparkRDDWriteClient(context(), writeConfig, Option.of(timelineService)); + // Both the write client and the table service client should use the same passed-in + // timeline server instance. + assertEquals(timelineService, writeClient.getTimelineServer().get()); + assertEquals(timelineService, writeClient.getTableServiceClient().getTimelineServer().get()); + // Write config should not be changed + assertEquals(writeConfig, writeClient.getConfig()); + timelineService.stop(); + } else { + writeClient = new SparkRDDWriteClient(context(), writeConfig); + // Only one timeline server should be instantiated, and the same timeline server + // should be used by both the write client and the table service client. + assertEquals( + writeClient.getTimelineServer(), + writeClient.getTableServiceClient().getTimelineServer()); + if (!enableEmbeddedTimelineServer) { + assertFalse(writeClient.getTimelineServer().isPresent()); + } + } + writeClient.close(); + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index f778c7cceac9c..686563ebfdd6c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieAvroRecord; @@ -34,10 +32,17 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieInsertException; +import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.List; @@ -79,36 +84,35 @@ public class TestTableSchemaEvolution extends HoodieClientTestBase { @Test public void testSchemaCompatibilityBasic() { - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA), + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA, false), "Same schema is compatible"); String reorderedSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + TIP_NESTED_SCHEMA + FARE_NESTED_SCHEMA + MAP_TYPE_SCHEMA + TRIP_SCHEMA_SUFFIX; - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema), + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema, false), "Reordered fields are compatible"); - assertTrue(isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA), + assertTrue(isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA, false), "Reordered fields are compatible"); String renamedSchema = TRIP_EXAMPLE_SCHEMA.replace("tip_history", "tip_future"); - // NOTE: That even though renames could be carried over as "column drop" and "column add" - // both of which are legitimate operations, no data carry-over will occur (exactly b/c - // it's an old column being dropped, and the new one being added) - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema), + assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema, false), + "Renaming fields is essentially: dropping old field, created a new one"); + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema, true), "Renaming fields is essentially: dropping old field, created a new one"); String renamedRecordSchema = TRIP_EXAMPLE_SCHEMA.replace("triprec", "triprec_renamed"); - assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema), + assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema, false), "Renamed record name is not compatible"); String swappedFieldSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA.replace("city_to_state", "fare") + FARE_NESTED_SCHEMA.replace("fare", "city_to_state") + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX; - assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema), + assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema, false), "Swapped fields are not compatible"); String typeChangeSchemaDisallowed = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA.replace("string", "boolean") + TRIP_SCHEMA_SUFFIX; - assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchemaDisallowed), + assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchemaDisallowed, false), "Incompatible field type change is not allowed"); // Array of allowed schema field type transitions @@ -119,10 +123,10 @@ public void testSchemaCompatibilityBasic() { for (String[] fieldChange : allowedFieldChanges) { String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[0]) + TRIP_SCHEMA_SUFFIX; String toSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA.replace("string", fieldChange[1]) + TRIP_SCHEMA_SUFFIX; - assertTrue(isSchemaCompatible(fromSchema, toSchema), + assertTrue(isSchemaCompatible(fromSchema, toSchema, false), "Compatible field type change is not allowed"); if (!fieldChange[0].equals("byte") && fieldChange[1].equals("byte")) { - assertFalse(isSchemaCompatible(toSchema, fromSchema), + assertFalse(isSchemaCompatible(toSchema, fromSchema, false), "Incompatible field type change is allowed"); } } @@ -130,32 +134,31 @@ public void testSchemaCompatibilityBasic() { // Names and aliases should match String fromSchema = TRIP_SCHEMA_PREFIX + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; String toSchema = TRIP_SCHEMA_PREFIX.replace("triprec", "new_triprec") + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX; - assertFalse(isSchemaCompatible(fromSchema, toSchema), "Field names should match"); - assertFalse(isSchemaCompatible(toSchema, fromSchema), "Field names should match"); + assertFalse(isSchemaCompatible(fromSchema, toSchema, false), "Field names should match"); + assertFalse(isSchemaCompatible(toSchema, fromSchema, false), "Field names should match"); - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED), + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, false), "Added field with default is compatible (Evolved Schema)"); String multipleAddedFieldSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + EXTRA_FIELD_SCHEMA.replace("new_field", "new_new_field") + TRIP_SCHEMA_SUFFIX; - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema), + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema, false), "Multiple added fields with defaults are compatible"); - assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, - TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA - + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA + TRIP_SCHEMA_SUFFIX), + assertFalse(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_WITHOUT_DEFAULT_SCHEMA + TRIP_SCHEMA_SUFFIX, false), "Added field without default and not nullable is not compatible (Evolved Schema)"); - assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, - TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA - + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA), + assertTrue(isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + + FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX + EXTRA_FIELD_NULLABLE_SCHEMA, false), "Added nullable field is compatible (Evolved Schema)"); } - @Test - public void testMORTable() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { tableType = HoodieTableType.MERGE_ON_READ; // Create the table @@ -165,7 +168,7 @@ public void testMORTable() throws Exception { .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); - HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA); + HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA, shouldAllowDroppedColumns); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA @@ -194,20 +197,26 @@ public void testMORTable() throws Exception { checkReadRecords("000", numRecords); // Insert with evolved schema (column dropped) is allowed - HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); + HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED, shouldAllowDroppedColumns); client = getHoodieWriteClient(hoodieDevolvedWriteConfig); final List failedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); // We cannot use insertBatch directly here because we want to insert records // with a evolved schema and insertBatch inserts records using the TRIP_EXAMPLE_SCHEMA. - writeBatch(client, "005", "004", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false); + try { + writeBatch(client, "005", "004", Option.empty(), "003", numRecords, + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false); + assertTrue(shouldAllowDroppedColumns); + } catch (HoodieInsertException e) { + assertFalse(shouldAllowDroppedColumns); + return; + } - // Update with evolved schema (column dropped) is allowed + // Update with evolved schema (column dropped) might be allowed depending on config set. updateBatch(hoodieDevolvedWriteConfig, client, "006", "005", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 2 * numRecords, 0); // Insert with an evolved scheme is allowed - HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); + HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); client = getHoodieWriteClient(hoodieEvolvedWriteConfig); // We cannot use insertBatch directly here because we want to insert records @@ -230,19 +239,28 @@ public void testMORTable() throws Exception { // Now try updating w/ the original schema (should succeed) client = getHoodieWriteClient(hoodieWriteConfig); - updateBatch(hoodieWriteConfig, client, "009", "008", Option.empty(), - initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9); + try { + updateBatch(hoodieWriteConfig, client, "009", "008", Option.empty(), + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9); + assertTrue(shouldAllowDroppedColumns); + } catch (HoodieUpsertException e) { + assertFalse(shouldAllowDroppedColumns); + } } - @Test - public void testCopyOnWriteTable() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Exception { // Create the table HoodieTableMetaClient.withPropertyBuilder() .fromMetaClient(metaClient) .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getHadoopConf(), metaClient.getBasePath()); - HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA).withRollbackUsingMarkers(false).build(); + HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA) + .withRollbackUsingMarkers(false) + .withAllowAutoEvolutionColumnDrop(shouldAllowDroppedColumns) + .build(); SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig); // Initial inserts with TRIP_EXAMPLE_SCHEMA @@ -266,11 +284,17 @@ public void testCopyOnWriteTable() throws Exception { checkReadRecords("000", numRecords); // Inserting records w/ new evolved schema (w/ tip column dropped) - HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); + HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED, shouldAllowDroppedColumns); client = getHoodieWriteClient(hoodieDevolvedWriteConfig); final List failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); - writeBatch(client, "004", "003", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false); + try { + writeBatch(client, "004", "003", Option.empty(), "003", numRecords, + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false); + assertTrue(shouldAllowDroppedColumns); + } catch (HoodieInsertException e) { + assertFalse(shouldAllowDroppedColumns); + return; + } // Updating records w/ new evolved schema updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(), @@ -278,7 +302,7 @@ public void testCopyOnWriteTable() throws Exception { numUpdateRecords, 2 * numRecords, 5); // Inserting with evolved schema is allowed - HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); + HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); client = getHoodieWriteClient(hoodieEvolvedWriteConfig); final List evolvedRecords = generateInsertsWithSchema("006", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); // We cannot use insertBatch directly here because we want to insert records @@ -299,9 +323,14 @@ public void testCopyOnWriteTable() throws Exception { // Now try updating w/ the original schema (should succeed) client = getHoodieWriteClient(hoodieWriteConfig); - updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), - initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, - numUpdateRecords, 3 * numRecords, 8); + try { + updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, + numUpdateRecords, 3 * numRecords, 8); + assertTrue(shouldAllowDroppedColumns); + } catch (HoodieUpsertException e) { + assertFalse(shouldAllowDroppedColumns); + } } private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException { @@ -362,8 +391,8 @@ private List convertToSchema(List records, String sc }).collect(Collectors.toList()); } - private HoodieWriteConfig getWriteConfig(String schema) { - return getWriteConfigBuilder(schema).build(); + private HoodieWriteConfig getWriteConfig(String schema, boolean shouldAllowDroppedColumns) { + return getWriteConfigBuilder(schema).withAllowAutoEvolutionColumnDrop(shouldAllowDroppedColumns).build(); } private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { @@ -373,8 +402,8 @@ private HoodieWriteConfig.Builder getWriteConfigBuilder(String schema) { .withAvroSchemaValidate(true); } - private static boolean isSchemaCompatible(String oldSchema, String newSchema) { - return AvroSchemaUtils.isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema)); + private static boolean isSchemaCompatible(String oldSchema, String newSchema, boolean shouldAllowDroppedColumns) { + return AvroSchemaUtils.isSchemaCompatible(new Schema.Parser().parse(oldSchema), new Schema.Parser().parse(newSchema), shouldAllowDroppedColumns); } @Override diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java index 78e711ed70129..99fb76650f945 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java @@ -18,12 +18,19 @@ package org.apache.hudi.client; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.util.Option; import org.junit.jupiter.api.Test; +import java.util.HashMap; +import java.util.Map; + import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -53,4 +60,89 @@ public void testSuccessRecordTracking() { assertTrue(status.getWrittenRecords().isEmpty()); assertEquals(2000, status.getTotalRecords()); } + + @Test + public void testSuccessWithEventTime() { + // test with empty eventTime + WriteStatus status = new WriteStatus(false, 1.0); + status.setStat(new HoodieWriteStat()); + for (int i = 0; i < 1000; i++) { + Map metadata = new HashMap<>(); + metadata.put(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY, ""); + status.markSuccess(mock(HoodieRecord.class), Option.of(metadata)); + } + assertEquals(1000, status.getTotalRecords()); + assertFalse(status.hasErrors()); + assertNull(status.getStat().getMaxEventTime()); + assertNull(status.getStat().getMinEventTime()); + + // test with null eventTime + status = new WriteStatus(false, 1.0); + status.setStat(new HoodieWriteStat()); + for (int i = 0; i < 1000; i++) { + Map metadata = new HashMap<>(); + metadata.put(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY, null); + status.markSuccess(mock(HoodieRecord.class), Option.of(metadata)); + } + assertEquals(1000, status.getTotalRecords()); + assertFalse(status.hasErrors()); + assertNull(status.getStat().getMaxEventTime()); + assertNull(status.getStat().getMinEventTime()); + + // test with seconds eventTime + status = new WriteStatus(false, 1.0); + status.setStat(new HoodieWriteStat()); + long minSeconds = 0L; + long maxSeconds = 0L; + for (int i = 0; i < 1000; i++) { + Map metadata = new HashMap<>(); + long eventTime = System.currentTimeMillis() / 1000; + if (i == 0) { + minSeconds = eventTime; + } else if (i == 999) { + maxSeconds = eventTime; + } + metadata.put(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY, String.valueOf(eventTime)); + status.markSuccess(mock(HoodieRecord.class), Option.of(metadata)); + } + assertEquals(1000, status.getTotalRecords()); + assertFalse(status.hasErrors()); + assertEquals(maxSeconds * 1000L, status.getStat().getMaxEventTime()); + assertEquals(minSeconds * 1000L, status.getStat().getMinEventTime()); + + // test with millis eventTime + status = new WriteStatus(false, 1.0); + status.setStat(new HoodieWriteStat()); + minSeconds = 0L; + maxSeconds = 0L; + for (int i = 0; i < 1000; i++) { + Map metadata = new HashMap<>(); + long eventTime = System.currentTimeMillis(); + if (i == 0) { + minSeconds = eventTime; + } else if (i == 999) { + maxSeconds = eventTime; + } + metadata.put(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY, String.valueOf(eventTime)); + status.markSuccess(mock(HoodieRecord.class), Option.of(metadata)); + } + assertEquals(1000, status.getTotalRecords()); + assertFalse(status.hasErrors()); + assertEquals(maxSeconds, status.getStat().getMaxEventTime()); + assertEquals(minSeconds, status.getStat().getMinEventTime()); + + // test with error format eventTime + status = new WriteStatus(false, 1.0); + status.setStat(new HoodieWriteStat()); + for (int i = 0; i < 1000; i++) { + Map metadata = new HashMap<>(); + metadata.put(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY, String.valueOf(i)); + status.markSuccess(mock(HoodieRecord.class), Option.of(metadata)); + } + assertEquals(1000, status.getTotalRecords()); + assertFalse(status.hasErrors()); + assertNull(status.getStat().getMaxEventTime()); + assertNull(status.getStat().getMinEventTime()); + + } } 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 8bfe13db9adf7..c36e3aed4e384 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 @@ -1962,6 +1962,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); @@ -2528,6 +2591,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; @@ -2576,7 +2643,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 d67e294e8b9b2..5c9697b840c60 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 @@ -47,6 +47,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieOperation; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -425,6 +426,15 @@ public void testDeduplicationOnInsert(boolean populateMetaFields) throws Excepti testDeduplication(SparkRDDWriteClient::insert, populateMetaFields); } + /** + * Test De-duplication behavior for HoodieWriteClient insert API. + */ + @ParameterizedTest + @MethodSource("populateMetaFieldsParams") + public void testDeduplicationKeepOperationFieldOnInsert(boolean populateMetaFields) throws Exception { + testDeduplicationKeepOperation(SparkRDDWriteClient::insert, populateMetaFields); + } + /** * Test De-duplication behavior for HoodieWriteClient bulk-insert API. */ @@ -511,6 +521,60 @@ private void testDeduplication( } } + /** + * Test Deduplication Logic for write function. + * + * @param writeFn One of HoodieWriteClient non-prepped write APIs + * @throws Exception in case of failure + */ + private void testDeduplicationKeepOperation( + Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean populateMetaFields) throws Exception { + String newCommitTime = "001"; + + String recordKey = UUID.randomUUID().toString(); + HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01"); + HoodieRecord recordOne = + new HoodieAvroRecord(keyOne, dataGen.generateRandomValue(keyOne, newCommitTime), HoodieOperation.INSERT); + + HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01"); + HoodieRecord recordTwo = + new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime), HoodieOperation.INSERT); + + // Same key and partition as keyTwo + HoodieRecord recordThree = + new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime), HoodieOperation.UPDATE_AFTER); + + HoodieData> records = HoodieJavaRDD.of( + jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1)); + HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withAllowOperationMetadataField(true) + .combineInput(true, true); + addConfigsForPopulateMetaFields(configBuilder, populateMetaFields); + HoodieWriteConfig writeConfig = configBuilder.build(); + + // Global dedup should be done based on recordKey only + HoodieIndex index = mock(HoodieIndex.class); + when(index.isGlobal()).thenReturn(true); + HoodieRecordMerger recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()); + int dedupParallelism = records.getNumPartitions() + 100; + HoodieData> dedupedRecsRdd = + (HoodieData>) HoodieWriteHelper.newInstance() + .deduplicateRecords(records, index, dedupParallelism, writeConfig.getSchema(), writeConfig.getProps(), recordMerger); + List> dedupedRecs = dedupedRecsRdd.collectAsList(); + assertEquals(dedupedRecs.get(0).getOperation(), recordThree.getOperation()); + + // Perform write-action and check + JavaRDD recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); + + try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig)) { + client.startCommitWithTime(newCommitTime); + List statuses = writeFn.apply(client, recordList, newCommitTime).collect(); + assertNoWriteErrors(statuses); + assertEquals(2, statuses.size()); + assertNodupesInPartition(statuses.stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream) + .collect(Collectors.toList())); + } + } + /** * Assert that there is no duplicate key at the partition level. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java index c623e13bddb09..be6a71426b619 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java @@ -85,27 +85,27 @@ public void testReadingMORTableWithoutBaseFile() throws Exception { // Do insert and updates thrice one after the other. // Insert String commitTime = HoodieActiveTimeline.createNewInstantTime(); - insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert, - false, false, 10, 10, 1, Option.empty()); + insertBatch(config, client, commitTime, "000", 100, SparkRDDWriteClient::insert, + false, false, 100, 100, 1, Option.empty()); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, - Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert, - false, false, 5, 10, 2, config.populateMetaFields()); + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 100, 2, config.populateMetaFields()); // Delete 5 records String prevCommitTime = commitTime; commitTime = HoodieActiveTimeline.createNewInstantTime(); deleteBatch(config, client, commitTime, prevCommitTime, - "000", 2, SparkRDDWriteClient::delete, false, false, - 0, 150); + "000", 25, SparkRDDWriteClient::delete, false, false, + 0, 100); // Verify all the records. metaClient.reloadActiveTimeline(); Map recordMap = GenericRecordValidationTestUtils.getRecordsMap(config, hadoopConf, dataGen); - assertEquals(8, recordMap.size()); + assertEquals(75, recordMap.size()); } @Test @@ -119,15 +119,15 @@ public void testCompactionOnMORTable() throws Exception { // Do insert and updates thrice one after the other. // Insert String commitTime = HoodieActiveTimeline.createNewInstantTime(); - insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert, - false, false, 10, 10, 1, Option.empty()); + insertBatch(config, client, commitTime, "000", 100, SparkRDDWriteClient::insert, + false, false, 100, 100, 1, Option.empty()); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, - Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert, - false, false, 5, 10, 2, config.populateMetaFields()); + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 5, 100, 2, config.populateMetaFields()); // Schedule and execute compaction. Option timeStamp = client.scheduleCompaction(Option.empty()); @@ -149,18 +149,20 @@ public void testLogCompactionOnMORTable() throws Exception { SparkRDDWriteClient client = getHoodieWriteClient(config); // First insert + int expectedTotalRecs = 100; String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); - insertBatch(config, client, newCommitTime, "000", 10, - SparkRDDWriteClient::insert, false, false, 10, 100, + insertBatch(config, client, newCommitTime, "000", expectedTotalRecs, + SparkRDDWriteClient::insert, false, false, expectedTotalRecs, expectedTotalRecs, 1, Option.empty()); String prevCommitTime = newCommitTime; for (int i = 0; i < 5; i++) { // Upsert newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + expectedTotalRecs += 50; updateBatch(config, client, newCommitTime, prevCommitTime, - Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert, - false, false, 50, 10, i + 2, config.populateMetaFields()); + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, expectedTotalRecs, i + 2, config.populateMetaFields()); prevCommitTime = newCommitTime; } @@ -173,9 +175,10 @@ public void testLogCompactionOnMORTable() throws Exception { for (int i = 0; i < 2; i++) { // Upsert newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + expectedTotalRecs += 50; updateBatch(config, client, newCommitTime, prevCommitTime, - Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert, - false, false, 50, 10, i + 8, config.populateMetaFields()); + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, expectedTotalRecs, i + 8, config.populateMetaFields()); prevCommitTime = newCommitTime; } String lastCommitBeforeLogCompaction = prevCommitTime; @@ -197,7 +200,7 @@ public void testLogCompactionOnMORTable() throws Exception { public void testLogCompactionOnMORTableWithoutBaseFile() throws Exception { HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder() .withLogCompactionBlocksThreshold("1") - .withLogRecordReaderScanV2("true") + .withEnableOptimizedLogBlocksScan("true") .build(); HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build(); @@ -205,23 +208,23 @@ public void testLogCompactionOnMORTableWithoutBaseFile() throws Exception { // First insert 10 records String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); - insertBatch(config, client, newCommitTime, "000", 10, - SparkRDDWriteClient::insert, false, false, 10, 10, + insertBatch(config, client, newCommitTime, "000", 100, + SparkRDDWriteClient::insert, false, false, 100, 100, 1, Option.of(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)); // Upsert 5 records String prevCommitTime = newCommitTime; newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, - Option.of(Arrays.asList(prevCommitTime)), "000", 5, SparkRDDWriteClient::upsert, - false, false, 5, 10, 2, config.populateMetaFields()); + Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, + false, false, 50, 100, 2, config.populateMetaFields()); prevCommitTime = newCommitTime; // Delete 3 records newCommitTime = HoodieActiveTimeline.createNewInstantTime(); deleteBatch(config, client, newCommitTime, prevCommitTime, - "000", 3, SparkRDDWriteClient::delete, false, false, - 0, 10); + "000", 30, SparkRDDWriteClient::delete, false, false, + 0, 70); String lastCommitBeforeLogCompaction = newCommitTime; // Schedule and execute compaction. @@ -249,7 +252,7 @@ public void testSchedulingLogCompactionAfterSchedulingCompaction() throws Except // First insert String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, - SparkRDDWriteClient::insert, false, false, 10, 100, + SparkRDDWriteClient::insert, false, false, 100, 100, 1, Option.empty()); String prevCommitTime = newCommitTime; @@ -257,7 +260,7 @@ public void testSchedulingLogCompactionAfterSchedulingCompaction() throws Except newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields()); // Schedule compaction Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); @@ -294,7 +297,7 @@ public void testSchedulingCompactionAfterSchedulingLogCompaction() throws Except // First insert String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, - SparkRDDWriteClient::insert, false, false, 10, 100, + SparkRDDWriteClient::insert, false, false, 100, 100, 1, Option.empty()); String prevCommitTime = newCommitTime; @@ -302,7 +305,7 @@ public void testSchedulingCompactionAfterSchedulingLogCompaction() throws Except newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields()); // Schedule log compaction Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty()); @@ -329,7 +332,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro // First insert. Here first file slice gets added to file group. String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, - SparkRDDWriteClient::insert, false, false, 10, 100, + SparkRDDWriteClient::insert, false, false, 100, 100, 1, Option.empty()); // Schedule and execute compaction. Here, second file slice gets added. @@ -342,7 +345,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields()); prevCommitTime = newCommitTime; // Schedule compaction. Third file slice gets added, compaction is not complete so base file is not created yet. @@ -355,7 +358,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields()); prevCommitTime = newCommitTime; if (i == 2) { // Since retain commits is 4 exactly after 6th completed commit there will be some files to be cleaned, @@ -384,7 +387,7 @@ public void testRollbackOnLogCompaction() throws Exception { // First insert String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, - SparkRDDWriteClient::insert, false, false, 10, 100, + SparkRDDWriteClient::insert, false, false, 100, 100, 1, Option.empty()); String prevCommitTime = newCommitTime; @@ -392,7 +395,7 @@ public void testRollbackOnLogCompaction() throws Exception { newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, - false, false, 10, 10, 4, config.populateMetaFields()); + false, false, 10, 100, 4, config.populateMetaFields()); prevCommitTime = newCommitTime; // Schedule and execute log-compaction but do not commit. @@ -418,7 +421,7 @@ public void testRollbackOnLogCompaction() throws Exception { newCommitTime = HoodieActiveTimeline.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, - false, false, 10, 10, 4, config.populateMetaFields()); + false, false, 10, 100, 4, config.populateMetaFields()); prevCommitTime = newCommitTime; // Complete log-compaction now. @@ -444,7 +447,7 @@ private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig confi .collect(Collectors.toList())) .withLatestInstantTime(instant) .withBufferSize(config.getMaxDFSStreamBufferSize()) - .withUseScanV2(true) + .withOptimizedLogBlocksScan(true) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); scanner.scan(true); @@ -458,7 +461,7 @@ private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig confi .collect(Collectors.toList())) .withLatestInstantTime(currentInstant) .withBufferSize(config.getMaxDFSStreamBufferSize()) - .withUseScanV2(true) + .withOptimizedLogBlocksScan(true) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); scanner2.scan(true); 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/client/functional/TestSavepointRestoreCopyOnWrite.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java new file mode 100644 index 0000000000000..8a71a01fda9fd --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.functional; + +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieSparkTable; +import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.Objects; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases for COPY_ON_WRITE table savepoint restore. + */ +@Tag("functional") +public class TestSavepointRestoreCopyOnWrite extends HoodieClientTestBase { + + /** + * Actions: C1, C2, savepoint C2, C3, C4, restore. + * Should go back to C2, + * C3 and C4 should be cleaned up. + */ + @Test + void testBasicRollback() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + String prevInstant = HoodieTimeline.INIT_INSTANT_TS; + final int numRecords = 10; + for (int i = 1; i <= 4; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2nd commit been rolled back + insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, + false, true, numRecords, numRecords * i, 1, Option.empty()); + prevInstant = newCommitTime; + if (i == 2) { + // trigger savepoint + savepointCommit = newCommitTime; + client.savepoint("user1", "Savepoint for 2nd commit"); + } + } + assertRowNumberEqualsTo(40); + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(20); + } + } + + /** + * The restore should roll back all the pending instants that are beyond the savepoint. + * + *

Actions: C1, C2, savepoint C2, C3, C4 inflight, restore. + * Should go back to C2, + * C3, C4 should be cleaned up. + */ + @Test + void testCleaningPendingInstants() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + String prevInstant = HoodieTimeline.INIT_INSTANT_TS; + final int numRecords = 10; + for (int i = 1; i <= 3; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2nd commit been rolled back + insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, + false, true, numRecords, numRecords * i, 1, Option.empty()); + prevInstant = newCommitTime; + if (i == 2) { + // trigger savepoint + savepointCommit = newCommitTime; + client.savepoint("user1", "Savepoint for 2nd commit"); + } + } + assertRowNumberEqualsTo(30); + // write another pending instant + insertBatchWithoutCommit(HoodieActiveTimeline.createNewInstantTime(), numRecords); + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(20); + } + } + + /** + * The rollbacks(either inflight or complete) beyond the savepoint should be cleaned. + * + *

Actions: C1, C2, savepoint C2, C3, C4 (RB_C3), C5, restore. + * Should go back to C2. + * C3, C4(RB_C3), C5 should be cleaned up. + * + *

Actions: C1, C2, savepoint C2, C3, C4 (RB_C3) inflight, restore. + * Should go back to C2. + * C3, C4 (RB_C3) should be cleaned up. + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCleaningRollbackInstants(boolean commitRollback) throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER) + // eager cleaning + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + String prevInstant = HoodieTimeline.INIT_INSTANT_TS; + final int numRecords = 10; + for (int i = 1; i <= 2; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2nd commit been rolled back + insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, + false, true, numRecords, numRecords * i, 1, Option.empty()); + prevInstant = newCommitTime; + if (i == 2) { + // trigger savepoint + savepointCommit = newCommitTime; + client.savepoint("user1", "Savepoint for 2nd commit"); + } + } + assertRowNumberEqualsTo(20); + // write another pending instant + insertBatchWithoutCommit(HoodieActiveTimeline.createNewInstantTime(), numRecords); + // rollback the pending instant + if (commitRollback) { + client.rollbackFailedWrites(); + } else { + HoodieInstant pendingInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction() + .lastInstant().orElseThrow(() -> new HoodieException("Pending instant does not exist")); + HoodieSparkTable.create(client.getConfig(), context) + .scheduleRollback(context, HoodieActiveTimeline.createNewInstantTime(), pendingInstant, false, true); + } + Option rollbackInstant = metaClient.reloadActiveTimeline().getRollbackTimeline().lastInstant().map(HoodieInstant::getTimestamp); + assertTrue(rollbackInstant.isPresent(), "The latest instant should be a rollback"); + // write another batch + insertBatch(hoodieWriteConfig, client, HoodieActiveTimeline.createNewInstantTime(), rollbackInstant.get(), numRecords, SparkRDDWriteClient::insert, + false, true, numRecords, numRecords * 3, 1, Option.empty()); + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(20); + } + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreMergeOnRead.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreMergeOnRead.java new file mode 100644 index 0000000000000..6c1dfe5d73420 --- /dev/null +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreMergeOnRead.java @@ -0,0 +1,248 @@ +/* + * 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.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.testutils.HoodieClientTestBase; + +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases for MERGE_ON_READ table savepoint restore. + */ +@Tag("functional") +public class TestSavepointRestoreMergeOnRead extends HoodieClientTestBase { + + /** + * Actions: DC1, DC2, DC3, savepoint DC3,(snapshot query) DC4, C5, DC6, DC7. restore to DC3. + * Should roll back DC5 and DC6. + * The latest file slice should be fully cleaned up, and rollback log appends for DC4 in first file slice. + * + *

For example file layout, + * FG1: + * BF1(DC1), LF1(DC2), LF2(DC3), LF3(DC4) + * BF5(C5), LF1(DC6), LF2(DC7) + * After restore, it becomes + * BF1(DC1), LF1(DC2), LF2(DC3), LF3(DC4), LF4(RB DC4) + * + *

Expected behaviors: + * snapshot query: total rec matches. + * checking the row count by updating columns in (val4,val5,val6, val7). + */ + @Test + void testCleaningDeltaCommits() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER) // eager cleaning + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(4) // the 4th delta_commit triggers compaction + .withInlineCompaction(true) + .build()) + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + final int numRecords = 10; + List baseRecordsToUpdate = null; + for (int i = 1; i <= 3; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2td commit been rolled back + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, numRecords); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.insert(writeRecords, newCommitTime); + if (i == 3) { + // trigger savepoint + savepointCommit = newCommitTime; + baseRecordsToUpdate = records; + client.savepoint("user1", "Savepoint for 3rd commit"); + } + } + + assertRowNumberEqualsTo(30); + + // write another 3 delta commits + for (int i = 1; i <= 3; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, Objects.requireNonNull(baseRecordsToUpdate, "The records to update should not be null")); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.upsert(writeRecords, newCommitTime); + if (i == 1) { + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent(), "A compaction plan should be scheduled"); + client.compact(compactionInstant.get()); + } + } + + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(30); + } + } + + /** + *

Actions: DC1, DC2, DC3, savepoint DC3, DC4, C5.pending, DC6, DC7, restore + * should roll back until DC3. + * + *

Expected behaviors: pending compaction after savepoint should also be cleaned, + * the latest file slice should be fully delete, for DC4 a rollback log append should be made. + */ + @Test + void testCleaningPendingCompaction() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER) // eager cleaning + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(4) // the 4th delta_commit triggers compaction + .withInlineCompaction(false) + .withScheduleInlineCompaction(true) + .build()) + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + final int numRecords = 10; + List baseRecordsToUpdate = null; + for (int i = 1; i <= 3; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2td commit been rolled back + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, numRecords); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.insert(writeRecords, newCommitTime); + if (i == 3) { + // trigger savepoint + savepointCommit = newCommitTime; + baseRecordsToUpdate = records; + client.savepoint("user1", "Savepoint for 3rd commit"); + } + } + + assertRowNumberEqualsTo(30); + + // write another 3 delta commits + for (int i = 1; i <= 3; i++) { + upsertBatch(writeClient, baseRecordsToUpdate); + if (i == 1) { + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent(), "A compaction plan should be scheduled"); + compactWithoutCommit(compactionInstant.get()); + } + } + + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(30); + } + } + + /** + * Actions: DC1, DC2, DC3, C4, savepoint C4, DC5, C6(RB_DC5), DC7, restore + * + *

Expected behaviors: should roll back DC5, C6 and DC6. + * No files will be cleaned up. Only rollback log appends. + */ + @Test + void testCleaningCompletedRollback() throws Exception { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER) // eager cleaning + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(3) // the 3rd delta_commit triggers compaction + .withInlineCompaction(false) + .withScheduleInlineCompaction(true) + .build()) + .withRollbackUsingMarkers(true) + .build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + String savepointCommit = null; + final int numRecords = 10; + List baseRecordsToUpdate = null; + for (int i = 1; i <= 2; i++) { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + // Write 4 inserts with the 2td commit been rolled back + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateInserts(newCommitTime, numRecords); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.insert(writeRecords, newCommitTime); + if (i == 2) { + baseRecordsToUpdate = records; + } + } + + // update to generate log files, then a valid compaction plan can be scheduled + upsertBatch(client, baseRecordsToUpdate); + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent(), "A compaction plan should be scheduled"); + client.compact(compactionInstant.get()); + savepointCommit = compactionInstant.get(); + client.savepoint("user1", "Savepoint for 3td commit"); + + assertRowNumberEqualsTo(20); + // write a delta_commit but does not commit + updateBatchWithoutCommit(HoodieActiveTimeline.createNewInstantTime(), Objects.requireNonNull(baseRecordsToUpdate, "The records to update should not be null")); + // rollback the delta_commit + assertTrue(writeClient.rollbackFailedWrites(), "The last delta_commit should be rolled back"); + + // another update + upsertBatch(writeClient, baseRecordsToUpdate); + + // restore + client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); + assertRowNumberEqualsTo(20); + } + } + + private void upsertBatch(SparkRDDWriteClient client, List baseRecordsToUpdate) throws IOException { + String newCommitTime = HoodieActiveTimeline.createNewInstantTime(); + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, Objects.requireNonNull(baseRecordsToUpdate, "The records to update should not be null")); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.upsert(writeRecords, newCommitTime); + } + + private void compactWithoutCommit(String compactionInstantTime) { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoCommit(false) // disable auto commit + .withRollbackUsingMarkers(true) + .build(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + JavaRDD statuses = (JavaRDD) client.compact(compactionInstantTime).getWriteStatuses(); + assertNoWriteErrors(statuses.collect()); + } + } + + @Override + protected HoodieTableType getTableType() { + return HoodieTableType.MERGE_ON_READ; + } +} diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java index 9dcd1d876a0e1..bbd9eaa4e02a5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java @@ -18,11 +18,14 @@ package org.apache.hudi.client.functional; +import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; import org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy; import org.apache.hudi.client.clustering.update.strategy.SparkConsistentBucketDuplicateUpdateStrategy; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieConsistentHashingMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; @@ -33,10 +36,11 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieArchivalConfig; +import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode; @@ -52,6 +56,7 @@ import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.io.DoubleWritable; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.mapred.JobConf; @@ -149,6 +154,59 @@ public void testResizing(boolean isSplit) throws IOException { }); } + /** + * Test running archival after clustering + * @throws IOException + */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testLoadMetadata(boolean isCommitFilePresent) throws IOException { + final int maxFileSize = 5120; + final int targetBucketNum = 14; + setup(maxFileSize); + writeClient.getConfig().setValue(HoodieCleanConfig.CLEANER_COMMITS_RETAINED.key(), "1"); + writeClient.getConfig().setValue(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP.key(), "4"); + writeClient.getConfig().setValue(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP.key(), "5"); + writeData(HoodieActiveTimeline.createNewInstantTime(), 2000, true); + String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); + writeClient.cluster(clusteringTime, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + metaClient = HoodieTableMetaClient.reload(metaClient); + final HoodieTable table = HoodieSparkTable.create(config, context, metaClient); + writeClient.clean(); + HoodieTimelineArchiver hoodieTimelineArchiver = new HoodieTimelineArchiver(writeClient.getConfig(), table); + hoodieTimelineArchiver.archiveIfRequired(context); + Arrays.stream(dataGen.getPartitionPaths()).forEach(p -> { + if (!isCommitFilePresent) { + Path metadataPath = FSUtils.getPartitionPath(table.getMetaClient().getHashingMetadataPath(), p); + try { + Arrays.stream(table.getMetaClient().getFs().listStatus(metadataPath)).forEach(fl -> { + if (fl.getPath().getName().contains(HoodieConsistentHashingMetadata.HASHING_METADATA_COMMIT_FILE_SUFFIX)) { + try { + // delete commit marker to test recovery job + table.getMetaClient().getFs().delete(fl.getPath()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + HoodieConsistentHashingMetadata metadata = HoodieSparkConsistentBucketIndex.loadMetadata(table, p).get(); + Assertions.assertEquals(targetBucketNum, metadata.getNodes().size()); + }); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + writeData(HoodieActiveTimeline.createNewInstantTime(), 10, true); + Assertions.assertEquals(2080, readRecords(dataGen.getPartitionPaths()).size()); + } + /** * 1. Test PARTITION_SORT mode, i.e., sort by the record key * 2. Test custom column sort diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java index 3ccdf1ec10690..eb61cb433120d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryExecutorInSpark.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; +import org.apache.hudi.common.util.queue.ExecutorType; import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -41,18 +42,21 @@ import scala.Tuple2; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TestBoundedInMemoryExecutorInSpark extends HoodieClientTestHarness { private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + private final HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withExecutorType(ExecutorType.BOUNDED_IN_MEMORY.name()) + .withWriteBufferLimitBytes(1024) + .build(false); + @BeforeEach public void setUp() throws Exception { initTestDataGenerator(); @@ -74,8 +78,6 @@ public void testExecutor() { final int recordNumber = 100; final List hoodieRecords = dataGen.generateInserts(instantTime, recordNumber); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); HoodieConsumer, Integer> consumer = new HoodieConsumer, Integer>() { @@ -94,8 +96,8 @@ public Integer finish() { BoundedInMemoryExecutor>, Integer> executor = null; try { - executor = new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + executor = new BoundedInMemoryExecutor(writeConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), getPreExecuteRunnable()); int result = executor.execute(); assertEquals(100, result); @@ -113,8 +115,6 @@ public Integer finish() { public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); HoodieConsumer, Integer> consumer = new HoodieConsumer, Integer>() { @@ -136,8 +136,8 @@ public Integer finish() { }; BoundedInMemoryExecutor>, Integer> executor = - new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), getPreExecuteRunnable()); + new BoundedInMemoryExecutor(writeConfig.getWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), getPreExecuteRunnable()); // Interrupt the current thread (therefore triggering executor to throw as soon as it // invokes [[get]] on the [[CompletableFuture]]) @@ -154,8 +154,6 @@ public Integer finish() { @Test public void testExecutorTermination() { - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024); Iterator unboundedRecordIter = new Iterator() { @Override public boolean hasNext() { @@ -181,8 +179,8 @@ public Integer finish() { }; BoundedInMemoryExecutor>, Integer> executor = - new BoundedInMemoryExecutor(hoodieWriteConfig.getWriteBufferLimitBytes(), unboundedRecordIter, - consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), + new BoundedInMemoryExecutor(writeConfig.getWriteBufferLimitBytes(), unboundedRecordIter, + consumer, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), getPreExecuteRunnable()); executor.shutdownNow(); boolean terminatedGracefully = executor.awaitTermination(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java index c9be18c9da3da..50ba44c5688c8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java @@ -27,9 +27,11 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SizeEstimator; import org.apache.hudi.common.util.queue.BoundedInMemoryQueue; +import org.apache.hudi.common.util.queue.ExecutorType; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.HoodieProducer; import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; @@ -54,7 +56,7 @@ import scala.Tuple2; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -65,6 +67,11 @@ public class TestBoundedInMemoryQueue extends HoodieClientTestHarness { private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + private final HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withExecutorType(ExecutorType.BOUNDED_IN_MEMORY.name()) + .withWriteBufferLimitBytes(1024) + .build(false); + @BeforeEach public void setUp() throws Exception { initTestDataGenerator(); @@ -85,7 +92,7 @@ public void testRecordReading() throws Exception { final int numRecords = 128; final List hoodieRecords = dataGen.generateInserts(instantTime, numRecords); final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(FileIOUtils.KB, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig)); // Produce Future resFuture = executorService.submit(() -> { new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue); @@ -125,7 +132,7 @@ public void testCompositeProducerRecordReading() throws Exception { final List> recs = new ArrayList<>(); final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(FileIOUtils.KB, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(FileIOUtils.KB, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig)); // Record Key to Map> keyToProducerAndIndexMap = new HashMap<>(); @@ -220,11 +227,11 @@ public void testMemoryLimitForBuffering() throws Exception { final int recordLimit = 5; final SizeEstimator sizeEstimator = new DefaultSizeEstimator<>(); HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA).apply((HoodieAvroRecord) hoodieRecords.get(0)); + getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig).apply(hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(genResult); final long memoryLimitInBytes = recordLimit * objSize; final BoundedInMemoryQueue queue = - new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig)); // Produce executorService.submit(() -> { @@ -269,7 +276,7 @@ public void testException() throws Exception { final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit HoodieLazyInsertIterable.HoodieInsertValueGenResult genResult = - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0)); + getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig).apply(hoodieRecords.get(0)); final long objSize = sizeEstimator.sizeEstimate(new Tuple2(genResult.getResult(), genResult.getResult().toIndexedRecord(HoodieTestDataGenerator.AVRO_SCHEMA, new Properties()))); final long memoryLimitInBytes = 4 * objSize; @@ -277,7 +284,7 @@ public void testException() throws Exception { // stops and throws // correct exception back. BoundedInMemoryQueue>> queue1 = - new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig)); // Produce Future resFuture = executorService.submit(() -> { @@ -305,7 +312,7 @@ public void testException() throws Exception { when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); BoundedInMemoryQueue>> queue2 = - new BoundedInMemoryQueue(memoryLimitInBytes, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + new BoundedInMemoryQueue(memoryLimitInBytes, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig)); // Produce Future res = executorService.submit(() -> { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java index 674aca7415395..19155f6b31877 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorExecutionInSpark.java @@ -18,13 +18,11 @@ package org.apache.hudi.execution; -import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.common.util.queue.DisruptorExecutor; +import org.apache.hudi.common.util.queue.ExecutorType; +import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.common.util.queue.WaitStrategyFactory; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -38,21 +36,23 @@ import java.util.ArrayList; import java.util.List; +import java.util.function.Function; -import scala.Tuple2; - -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TestDisruptorExecutionInSpark extends HoodieClientTestHarness { private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + + private final HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withExecutorType(ExecutorType.DISRUPTOR.name()) + .withWriteExecutorDisruptorWriteBufferLimitBytes(8) + .build(false); + @BeforeEach public void setUp() throws Exception { initTestDataGenerator(); @@ -75,16 +75,14 @@ public void testExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 128); final List consumedRecords = new ArrayList<>(); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(8)); - HoodieConsumer, Integer> consumer = - new HoodieConsumer, Integer>() { + HoodieConsumer consumer = + new HoodieConsumer() { private int count = 0; @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { - consumedRecords.add(record.getResult()); + public void consume(HoodieRecord record) { + consumedRecords.add(record); count++; } @@ -93,11 +91,11 @@ public Integer finish() { return count; } }; - DisruptorExecutor>, Integer> exec = null; + DisruptorExecutor exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + exec = new DisruptorExecutor<>(writeConfig.getWriteExecutorDisruptorWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + Function.identity(), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(128, result); @@ -123,15 +121,15 @@ public Integer finish() { public void testInterruptExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 100); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(1024)); - HoodieConsumer, Integer> consumer = - new HoodieConsumer, Integer>() { + HoodieConsumer consumer = + new HoodieConsumer() { @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) { + public void consume(HoodieRecord record) { try { - Thread.currentThread().wait(); + synchronized (this) { + wait(); + } } catch (InterruptedException ie) { // ignore here } @@ -143,9 +141,9 @@ public Integer finish() { } }; - DisruptorExecutor>, Integer> - executor = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + DisruptorExecutor + executor = new DisruptorExecutor<>(1024, hoodieRecords.iterator(), consumer, + Function.identity(), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); try { Thread.currentThread().interrupt(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java index 2ff9d02926b01..4c8e0dac27dbc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestDisruptorMessageQueue.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.DisruptorExecutor; import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.ExecutorType; import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer; import org.apache.hudi.common.util.queue.HoodieConsumer; import org.apache.hudi.common.util.queue.HoodieProducer; @@ -56,17 +57,20 @@ import java.util.stream.IntStream; import static org.apache.hudi.exception.ExceptionUtil.getRootCause; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; +import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformer; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class TestDisruptorMessageQueue extends HoodieClientTestHarness { private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); + private final HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() + .withExecutorType(ExecutorType.DISRUPTOR.name()) + .withWriteExecutorDisruptorWriteBufferLimitBytes(16) + .build(false); + @BeforeEach public void setUp() throws Exception { initTestDataGenerator(); @@ -108,8 +112,6 @@ public void testRecordReading() throws Exception { } }); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(16)); HoodieConsumer, Integer> consumer = new HoodieConsumer, Integer>() { @@ -137,8 +139,8 @@ public Integer finish() { DisruptorExecutor>, Integer> exec = null; try { - exec = new DisruptorExecutor(hoodieWriteConfig.getDisruptorWriteBufferSize(), hoodieRecords.iterator(), consumer, - getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + exec = new DisruptorExecutor(writeConfig.getWriteExecutorDisruptorWriteBufferLimitBytes(), hoodieRecords.iterator(), consumer, + getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); int result = exec.execute(); // It should buffer and write 100 records assertEquals(100, result); @@ -167,8 +169,8 @@ public void testCompositeProducerRecordReading() throws Exception { final List> recs = new ArrayList<>(); final DisruptorMessageQueue queue = - new DisruptorMessageQueue(Option.of(1024), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), - Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { + new DisruptorMessageQueue(1024, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), + "BLOCKING_WAIT", numProducers, new Runnable() { @Override public void run() { // do nothing. @@ -282,8 +284,8 @@ public void testException() throws Exception { final int numProducers = 40; final DisruptorMessageQueue queue = - new DisruptorMessageQueue(Option.of(1024), getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), - Option.of("BLOCKING_WAIT"), numProducers, new Runnable() { + new DisruptorMessageQueue(1024, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), + "BLOCKING_WAIT", numProducers, new Runnable() { @Override public void run() { // do nothing. @@ -324,9 +326,9 @@ public Integer finish() { } }; - DisruptorExecutor>, Integer> exec = new DisruptorExecutor(Option.of(1024), - producers, consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA), - Option.of(WaitStrategyFactory.DEFAULT_STRATEGY), getPreExecuteRunnable()); + DisruptorExecutor>, Integer> exec = new DisruptorExecutor(1024, + producers, consumer, getTransformer(HoodieTestDataGenerator.AVRO_SCHEMA, writeConfig), + WaitStrategyFactory.DEFAULT_STRATEGY, getPreExecuteRunnable()); final Throwable thrown = assertThrows(HoodieException.class, exec::execute, "exception is expected"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSimpleExecutionInSpark.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSimpleExecutionInSpark.java index bbc85efd376a6..830577463b299 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSimpleExecutionInSpark.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSimpleExecutionInSpark.java @@ -18,8 +18,6 @@ package org.apache.hudi.execution; -import static org.apache.hudi.execution.HoodieLazyInsertIterable.getCloningTransformer; - import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieRecord; @@ -27,8 +25,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.queue.HoodieConsumer; -import org.apache.hudi.common.util.queue.SimpleHoodieExecutor; -import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.common.util.queue.SimpleExecutor; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.testutils.HoodieClientTestHarness; import org.junit.jupiter.api.AfterEach; @@ -41,16 +38,13 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; - -import scala.Tuple2; +import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -public class TestSimpleExecutionInSpark extends HoodieClientTestHarness { +public class TestSimpleExecutionInSpark extends HoodieClientTestHarness { private final String instantTime = HoodieActiveTimeline.createNewInstantTime(); @@ -71,15 +65,13 @@ public void testExecutor() { final List hoodieRecords = dataGen.generateInserts(instantTime, 128); final List consumedRecords = new ArrayList<>(); - HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class); - when(hoodieWriteConfig.getDisruptorWriteBufferSize()).thenReturn(Option.of(8)); - HoodieConsumer, Integer> consumer = - new HoodieConsumer, Integer>() { + HoodieConsumer consumer = + new HoodieConsumer() { private int count = 0; @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) throws Exception { - consumedRecords.add(record.getResult()); + public void consume(HoodieRecord record) throws Exception { + consumedRecords.add(record); count++; } @@ -88,10 +80,10 @@ public Integer finish() { return count; } }; - SimpleHoodieExecutor>, Integer> exec = null; + SimpleExecutor exec = null; try { - exec = new SimpleHoodieExecutor(hoodieRecords.iterator(), consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + exec = new SimpleExecutor<>(hoodieRecords.iterator(), consumer, Function.identity()); int result = exec.execute(); // It should buffer and write 128 records @@ -133,16 +125,16 @@ public void testRecordReading() { } }); - HoodieConsumer, Integer> consumer = - new HoodieConsumer, Integer>() { + HoodieConsumer consumer = + new HoodieConsumer() { private int count = 0; @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult record) throws Exception { + public void consume(HoodieRecord record) throws Exception { count++; - afterRecord.add((HoodieAvroRecord) record.getResult()); + afterRecord.add((HoodieAvroRecord) record); try { - IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record.getResult()) + IndexedRecord indexedRecord = (IndexedRecord)((HoodieAvroRecord) record) .getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get(); afterIndexedRecord.add(indexedRecord); } catch (IOException e) { @@ -156,10 +148,10 @@ public Integer finish() { } }; - SimpleHoodieExecutor>, Integer> exec = null; + SimpleExecutor exec = null; try { - exec = new SimpleHoodieExecutor(hoodieRecords.iterator(), consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + exec = new SimpleExecutor<>(hoodieRecords.iterator(), consumer, Function.identity()); int result = exec.execute(); assertEquals(100, result); @@ -186,14 +178,13 @@ public void testException() { List pRecs = dataGen.generateInserts(instantTime, numRecords); InnerIterator iterator = new InnerIterator(pRecs.iterator(), errorMessage, numRecords / 10); - HoodieConsumer, Integer> consumer = - new HoodieConsumer, Integer>() { + HoodieConsumer consumer = + new HoodieConsumer() { int count = 0; @Override - public void consume(HoodieLazyInsertIterable.HoodieInsertValueGenResult payload) throws Exception { + public void consume(HoodieRecord payload) throws Exception { // Read recs and ensure we have covered all producer recs. - final HoodieRecord rec = payload.getResult(); count++; } @@ -203,8 +194,8 @@ public Integer finish() { } }; - SimpleHoodieExecutor>, Integer> exec = - new SimpleHoodieExecutor(iterator, consumer, getCloningTransformer(HoodieTestDataGenerator.AVRO_SCHEMA)); + SimpleExecutor exec = + new SimpleExecutor<>(iterator, consumer, Function.identity()); final Throwable thrown = assertThrows(HoodieException.class, exec::execute, "exception is expected"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java index 11420c180eb09..c740fb38afc05 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.FlatLists; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -210,7 +211,7 @@ public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode, @Test public void testCustomColumnSortPartitioner() { - String sortColumnString = "rider"; + String sortColumnString = "begin_lat"; String[] sortColumns = sortColumnString.split(","); Comparator> columnComparator = getCustomColumnComparator(HoodieTestDataGenerator.AVRO_SCHEMA, sortColumns); @@ -238,16 +239,19 @@ private Comparator> getCustomColumnC Comparator> comparator = Comparator.comparing(record -> { try { GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema).get(); - StringBuilder sb = new StringBuilder(); + List keys = new ArrayList<>(); for (String col : sortColumns) { - sb.append(genericRecord.get(col)); + keys.add(genericRecord.get(col)); } - - return sb.toString(); + return keys; } catch (IOException e) { throw new HoodieIOException("unable to read value for " + sortColumns); } - }); + }, (o1, o2) -> { + FlatLists.ComparableList values1 = FlatLists.ofComparableArray(o1.toArray()); + FlatLists.ComparableList values2 = FlatLists.ofComparableArray(o2.toArray()); + return values1.compareTo(values2); + }); return comparator; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java index 6332e00ba6706..f4e99f4670397 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitionerForRows.java @@ -48,6 +48,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Unit tests {@link BulkInsertPartitioner}s with Rows. @@ -172,9 +173,14 @@ private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner return; } Dataset actualRecords = (Dataset) partitioner.repartitionRecords(rows, numPartitions); - assertEquals( - enforceNumOutputPartitions ? numPartitions : rows.rdd().getNumPartitions(), - actualRecords.rdd().getNumPartitions()); + if (isGloballySorted) { + // For GLOBAL_SORT, `df.sort` may generate smaller number of partitions than the specified parallelism + assertTrue(actualRecords.rdd().getNumPartitions() <= numPartitions); + } else { + assertEquals( + enforceNumOutputPartitions ? numPartitions : rows.rdd().getNumPartitions(), + actualRecords.rdd().getNumPartitions()); + } List collectedActualRecords = actualRecords.collectAsList(); if (isGloballySorted) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java index e946450c90426..fe178839c0dc5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java @@ -18,6 +18,7 @@ package org.apache.hudi.index.bloom; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.util.collection.Pair; import org.junit.jupiter.api.Test; @@ -36,33 +37,37 @@ public class TestBucketizedBloomCheckPartitioner { @Test public void testAssignmentCorrectness() { - Map fileToComparisons = new HashMap() { + HoodieFileGroupId fg1 = new HoodieFileGroupId("p1", "f1"); + HoodieFileGroupId fg2 = new HoodieFileGroupId("p1", "f2"); + HoodieFileGroupId fg3 = new HoodieFileGroupId("p1", "f3"); + + Map fileToComparisons = new HashMap() { { - put("f1", 40L); - put("f2", 35L); - put("f3", 20L); + put(fg1, 40L); + put(fg2, 35L); + put(fg3, 20L); } }; BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(4, fileToComparisons, 10); - Map> assignments = p.getFileGroupToPartitions(); - assertEquals(4, assignments.get("f1").size(), "f1 should have 4 buckets"); - assertEquals(4, assignments.get("f2").size(), "f2 should have 4 buckets"); - assertEquals(2, assignments.get("f3").size(), "f3 should have 2 buckets"); - assertArrayEquals(new Integer[] {0, 0, 1, 3}, assignments.get("f1").toArray(), "f1 spread across 3 partitions"); - assertArrayEquals(new Integer[] {1, 2, 2, 0}, assignments.get("f2").toArray(), "f2 spread across 3 partitions"); - assertArrayEquals(new Integer[] {3, 1}, assignments.get("f3").toArray(), "f3 spread across 2 partitions"); + Map> assignments = p.getFileGroupToPartitions(); + assertEquals(4, assignments.get(fg1).size(), "f1 should have 4 buckets"); + assertEquals(4, assignments.get(fg2).size(), "f2 should have 4 buckets"); + assertEquals(2, assignments.get(fg3).size(), "f3 should have 2 buckets"); + assertArrayEquals(new Integer[] {0, 0, 1, 3}, assignments.get(fg1).toArray(), "f1 spread across 3 partitions"); + assertArrayEquals(new Integer[] {2, 2, 3, 1}, assignments.get(fg2).toArray(), "f2 spread across 3 partitions"); + assertArrayEquals(new Integer[] {1, 0}, assignments.get(fg3).toArray(), "f3 spread across 2 partitions"); } @Test public void testUniformPacking() { // evenly distribute 10 buckets/file across 100 partitions - Map comparisons1 = new HashMap() { + Map comparisons1 = new HashMap() { { - IntStream.range(0, 10).forEach(f -> put("f" + f, 100L)); + IntStream.range(0, 10).forEach(f -> put(new HoodieFileGroupId("p1", "f" + f), 100L)); } }; BucketizedBloomCheckPartitioner partitioner = new BucketizedBloomCheckPartitioner(100, comparisons1, 10); - Map> assignments = partitioner.getFileGroupToPartitions(); + Map> assignments = partitioner.getFileGroupToPartitions(); assignments.forEach((key, value) -> assertEquals(10, value.size())); Map partitionToNumBuckets = assignments.entrySet().stream().flatMap(e -> e.getValue().stream().map(p -> Pair.of(p, e.getKey()))) @@ -72,9 +77,9 @@ public void testUniformPacking() { @Test public void testNumPartitions() { - Map comparisons1 = new HashMap() { + Map comparisons1 = new HashMap() { { - IntStream.range(0, 10).forEach(f -> put("f" + f, 100L)); + IntStream.range(0, 10).forEach(f -> put(new HoodieFileGroupId("p1", "f" + f), 100L)); } }; BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(10000, comparisons1, 10); @@ -83,15 +88,15 @@ public void testNumPartitions() { @Test public void testGetPartitions() { - Map comparisons1 = new HashMap() { + Map comparisons1 = new HashMap() { { - IntStream.range(0, 100000).forEach(f -> put("f" + f, 100L)); + IntStream.range(0, 100000).forEach(f -> put(new HoodieFileGroupId("p1", "f" + f), 100L)); } }; BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(1000, comparisons1, 10); IntStream.range(0, 100000).forEach(f -> { - int partition = p.getPartition(Pair.of("f" + f, "value")); + int partition = p.getPartition(Pair.of(new HoodieFileGroupId("p1", "f" + f), "value")); assertTrue(0 <= partition && partition <= 1000, "partition is out of range: " + partition); }); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java index 5be4e4ce624a3..3c906062c16ed 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.WriteOperationType; @@ -35,7 +36,6 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndexUtils; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; @@ -259,12 +259,14 @@ public void testRangePruning( jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t); - List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( - index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect(); + List> comparisonKeyList = + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD)).collectAsList(); assertEquals(10, comparisonKeyList.size()); Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getLeft, Collectors.toList()))); + .collect( + Collectors.groupingBy(t -> t.getRight(), + Collectors.mapping(t -> t.getLeft().getFileId(), Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002"))); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java index 3ad8952feea84..2577f9ba28433 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java @@ -21,6 +21,7 @@ import org.apache.hudi.client.functional.TestHoodieMetadataBase; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.WriteOperationType; @@ -29,7 +30,6 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaPairRDD; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; @@ -201,9 +201,9 @@ public void testExplodeRecordRDDWithFileComparisons() { jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t); - List> comparisonKeyList = HoodieJavaRDD.getJavaRDD( - index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, - HoodieJavaPairRDD.of(partitionRecordKeyPairRDD))).collect(); + List> comparisonKeyList = + index.explodeRecordsWithFileComparisons(partitionToFileIndexInfo, HoodieJavaPairRDD.of(partitionRecordKeyPairRDD)) + .collectAsList(); /* * expecting: f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} f1, HoodieKey { recordKey=003 @@ -216,7 +216,7 @@ public void testExplodeRecordRDDWithFileComparisons() { assertEquals(10, comparisonKeyList.size()); Map> recordKeyToFileComps = comparisonKeyList.stream() - .collect(Collectors.groupingBy(t -> t.getRight().getRecordKey(), Collectors.mapping(Pair::getKey, Collectors.toList()))); + .collect(Collectors.groupingBy(t -> t.getRight(), Collectors.mapping(t -> t.getLeft().getFileId(), Collectors.toList()))); assertEquals(4, recordKeyToFileComps.size()); assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002"))); 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 6ac0a4907326a..4320b13662e8f 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 @@ -1400,6 +1400,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/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index 7577ba8c833a3..954bab910bc81 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -28,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; @@ -48,6 +49,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; @@ -56,6 +58,7 @@ import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV1MigrationHandler; +import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -65,6 +68,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -1088,6 +1092,97 @@ public void testRerunFailedClean(boolean simulateMetadataFailure) throws Excepti assertTrue(testTable.baseFileExists(p1, "00000000000001", file1P1C0)); } + /** Test if cleaner will fallback to full clean if commit for incremental clean is archived. */ + @Test + public void testIncrementalFallbackToFullClean() throws Exception { + HoodieWriteConfig config = HoodieWriteConfig.newBuilder() + .withCleanConfig( + HoodieCleanConfig.newBuilder() + .retainCommits(1) + .withIncrementalCleaningMode(true) + .build()) + .withMetadataConfig( + HoodieMetadataConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withArchivalConfig( + HoodieArchivalConfig.newBuilder() + .archiveCommitsWith(3, 4).build()) + .withMarkersType(MarkerType.DIRECT.name()) + .withPath(basePath) + .build(); + HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context); + HoodieTestTable testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter); + + String p1 = "part_1"; + String p2 = "part_2"; + testTable.withPartitionMetaFiles(p1, p2); + + // add file partition "part_1" + String file1P1 = UUID.randomUUID().toString(); + String file2P1 = UUID.randomUUID().toString(); + Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p1, CollectionUtils.createImmutableList(file1P1, file2P1)); + } + }); + commitWithMdt("1", part1ToFileId, testTable, metadataWriter); + commitWithMdt("2", part1ToFileId, testTable, metadataWriter); + + + // add clean instant + HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), + "", "", new HashMap<>(), CleanPlanV2MigrationHandler.VERSION, new HashMap<>(), new ArrayList<>()); + HoodieCleanMetadata cleanMeta = new HoodieCleanMetadata("", 0L, 0, + "2", "", new HashMap<>(), CleanPlanV2MigrationHandler.VERSION, new HashMap<>()); + testTable.addClean("3", cleanerPlan, cleanMeta); + + // add file in partition "part_2" + String file3P2 = UUID.randomUUID().toString(); + String file4P2 = UUID.randomUUID().toString(); + Map> part2ToFileId = Collections.unmodifiableMap(new HashMap>() { + { + put(p2, CollectionUtils.createImmutableList(file3P2, file4P2)); + } + }); + commitWithMdt("3", part2ToFileId, testTable, metadataWriter); + commitWithMdt("4", part2ToFileId, testTable, metadataWriter); + + // empty commits + testTable.addCommit("5"); + + // archive commit 1, 2 + new HoodieTimelineArchiver<>(config, HoodieSparkTable.create(config, context, metaClient)) + .archiveIfRequired(context, false); + + runCleaner(config); + assertFalse(testTable.baseFileExists(p1, "1", file1P1), "Clean old FileSlice in p1 by fallback to full clean"); + assertFalse(testTable.baseFileExists(p1, "1", file2P1), "Clean old FileSlice in p1 by fallback to full clean"); + assertFalse(testTable.baseFileExists(p2, "3", file3P2), "Clean old FileSlice in p2"); + assertFalse(testTable.baseFileExists(p2, "3", file4P2), "Clean old FileSlice in p2"); + assertTrue(testTable.baseFileExists(p1, "2", file1P1), "Latest FileSlice exists"); + assertTrue(testTable.baseFileExists(p1, "2", file2P1), "Latest FileSlice exists"); + assertTrue(testTable.baseFileExists(p2, "4", file3P2), "Latest FileSlice exists"); + assertTrue(testTable.baseFileExists(p2, "4", file4P2), "Latest FileSlice exists"); + } + + private void commitWithMdt(String instantTime, Map> partToFileId, + HoodieTestTable testTable, HoodieTableMetadataWriter metadataWriter) throws Exception { + HoodieCommitMetadata commitMeta = generateCommitMetadata(instantTime, partToFileId); + testTable.addInflightCommit(instantTime); + partToFileId.forEach((key, value) -> { + try { + testTable.withBaseFilesInPartition(key, value.toArray(new String[0])); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + metadataWriter.update(commitMeta, instantTime, false); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), + Option.of(commitMeta.toJsonString().getBytes(StandardCharsets.UTF_8))); + metaClient = HoodieTableMetaClient.reload(metaClient); + } + /** * Common test method for validating pending compactions. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 3dfcf24ff5981..b49092f0d5163 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -65,6 +65,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.storage.StorageLevel; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -98,6 +99,11 @@ void setUp(Properties props) throws IOException { dataGen = new HoodieTestDataGenerator(); } + @BeforeEach + void beforeEach() { + jsc().getPersistentRDDs().values().forEach(JavaRDD::unpersist); + } + // Check if record level metadata is aggregated properly at the end of write. @Test public void testMetadataAggregateFromWriteStatus() throws Exception { @@ -404,6 +410,7 @@ public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields) th public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception { HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY) .withAvroSchemaValidate(false) + .withAllowAutoEvolutionColumnDrop(true) .withAutoCommit(false) .build(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index c6cd554e28963..c0e62631664a8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -21,7 +21,9 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; @@ -37,7 +39,6 @@ import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieMemoryConfig; -import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.index.HoodieIndex; @@ -53,6 +54,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; @@ -113,7 +115,7 @@ public void testCompactionOnCopyOnWriteFail() throws Exception { } @Test - public void testCompactionEmpty() throws Exception { + public void testCompactionEmpty() { HoodieWriteConfig config = getConfig(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient); @@ -169,41 +171,45 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { writeClient.insert(recordsRDD, newCommitTime).collect(); // Update all the 100 records - HoodieTable table = HoodieSparkTable.create(config, context); newCommitTime = "101"; + updateRecords(config, newCommitTime, records); - List updatedRecords = dataGen.generateUpdates(newCommitTime, records); - JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); - HoodieIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); - JavaRDD updatedTaggedRecordsRDD = tagLocation(index, updatedRecordsRDD, table); + assertLogFilesNumEqualsTo(config, 1); + String compactionInstantTime = "102"; + HoodieData result = compact(writeClient, compactionInstantTime); + + verifyCompaction(result); + } + } + + @Test + public void testSpillingWhenCompaction() throws Exception { + // insert 100 records + HoodieWriteConfig config = getConfigBuilder() + .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build()) + .withMemoryConfig(HoodieMemoryConfig.newBuilder() + .withMaxMemoryMaxSize(1L, 1L).build()) // force spill + .build(); + try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) { + String newCommitTime = "100"; writeClient.startCommitWithTime(newCommitTime); - writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); - metaClient.reloadActiveTimeline(); - - // Verify that all data file has one log file - table = HoodieSparkTable.create(config, context); - for (String partitionPath : dataGen.getPartitionPaths()) { - List groupedLogFiles = - table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); - for (FileSlice fileSlice : groupedLogFiles) { - assertEquals(1, fileSlice.getLogFiles().count(), "There should be 1 log file written for every data file"); - } - } - // Do a compaction - table = HoodieSparkTable.create(config, context); - String compactionInstantTime = "102"; - table.scheduleCompaction(context, compactionInstantTime, Option.empty()); - table.getMetaClient().reloadActiveTimeline(); - HoodieData result = (HoodieData) table.compact( - context, compactionInstantTime).getWriteStatuses(); - - // Verify that all partition paths are present in the WriteStatus result - for (String partitionPath : dataGen.getPartitionPaths()) { - List writeStatuses = result.collectAsList(); - assertTrue(writeStatuses.stream() - .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)).count() > 0); + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD recordsRDD = jsc.parallelize(records, 1); + writeClient.insert(recordsRDD, newCommitTime).collect(); + + // trigger 2 updates following with compaction + for (int i = 1; i < 5; i += 2) { + // Update all the 100 records + newCommitTime = "10" + i; + updateRecords(config, newCommitTime, records); + + assertLogFilesNumEqualsTo(config, 1); + + HoodieData result = compact(writeClient, "10" + (i + 1)); + + verifyCompaction(result); } } } @@ -212,4 +218,52 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; } + + private void updateRecords(HoodieWriteConfig config, String newCommitTime, List records) throws IOException { + HoodieTable table = HoodieSparkTable.create(config, context); + List updatedRecords = dataGen.generateUpdates(newCommitTime, records); + JavaRDD updatedRecordsRDD = jsc.parallelize(updatedRecords, 1); + HoodieIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance()); + JavaRDD updatedTaggedRecordsRDD = tagLocation(index, updatedRecordsRDD, table); + + writeClient.startCommitWithTime(newCommitTime); + writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect(); + metaClient.reloadActiveTimeline(); + } + + /** + * Verify that all data file has {@code expected} number of log files. + * + * @param config The writer config + * @param expected The expected number of log files + */ + private void assertLogFilesNumEqualsTo(HoodieWriteConfig config, int expected) { + HoodieTable table = HoodieSparkTable.create(config, context); + for (String partitionPath : dataGen.getPartitionPaths()) { + List groupedLogFiles = + table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList()); + for (FileSlice fileSlice : groupedLogFiles) { + assertEquals(expected, fileSlice.getLogFiles().count(), "There should be " + expected + " log file written for every data file"); + } + } + } + + /** + * Do a compaction. + */ + private HoodieData compact(SparkRDDWriteClient writeClient, String compactionInstantTime) { + writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); + JavaRDD writeStatusJavaRDD = (JavaRDD) writeClient.compact(compactionInstantTime).getWriteStatuses(); + return HoodieListData.eager(writeStatusJavaRDD.collect()); + } + + /** + * Verify that all partition paths are present in the WriteStatus result. + */ + private void verifyCompaction(HoodieData result) { + for (String partitionPath : dataGen.getPartitionPaths()) { + List writeStatuses = result.collectAsList(); + assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))); + } + } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java index 927f8f3c24b82..47e93ab2e94f0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java @@ -145,7 +145,7 @@ public void testCopyOnWriteRollback(boolean useFileListingMetadata) throws Excep HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, writeConfig)) { // rollback 2nd commit and ensure stats reflect the info. - List stats = testRun(useFileListingMetadata, writeConfig, writeClient); + List stats = testUpdateAndRollback(useFileListingMetadata, writeConfig, writeClient); assertEquals(3, stats.size()); for (HoodieRollbackStat stat : stats) { @@ -172,7 +172,7 @@ public void testMergeOnReadRollback(boolean useFileListingMetadata) throws Excep try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, writeConfig)) { // rollback 2nd commit and ensure stats reflect the info. - List stats = testRun(useFileListingMetadata, writeConfig, writeClient); + List stats = testUpdateAndRollback(useFileListingMetadata, writeConfig, writeClient); assertEquals(3, stats.size()); for (HoodieRollbackStat stat : stats) { @@ -184,7 +184,55 @@ public void testMergeOnReadRollback(boolean useFileListingMetadata) throws Excep } } - private List testRun(boolean useFileListingMetadata, HoodieWriteConfig writeConfig, SparkRDDWriteClient writeClient) { + @ParameterizedTest(name = TEST_NAME_WITH_PARAMS) + @MethodSource("configParams") + public void testMergeOnReadRollbackDeletesFirstAppendFiles(boolean useFileListingMetadata) throws Exception { + // init MERGE_ON_READ_TABLE + tearDown(); + tableType = HoodieTableType.MERGE_ON_READ; + setUp(); + + HoodieWriteConfig writeConfig = getConfigBuilder().withRollbackUsingMarkers(true).withAutoCommit(false) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(useFileListingMetadata).build()) + .withPath(basePath).build(); + + HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, writeConfig)) { + + // rollback 2nd commit and ensure stats reflect the info. + List stats = testInsertAndRollback(writeClient); + + assertEquals(3, stats.size()); + for (HoodieRollbackStat stat : stats) { + assertEquals(1, stat.getSuccessDeleteFiles().size()); + assertEquals(0, stat.getFailedDeleteFiles().size()); + assertEquals(0, stat.getCommandBlocksCount().size()); + stat.getCommandBlocksCount().forEach((fileStatus, len) -> assertTrue(fileStatus.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension()))); + } + } + } + + private List testInsertAndRollback(SparkRDDWriteClient writeClient) { + String newCommitTime = "001"; + writeClient.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, 100); + JavaRDD writeStatuses = writeClient.insert(jsc.parallelize(records, 1), newCommitTime); + writeClient.commit(newCommitTime, writeStatuses); + + writeStatuses.collect(); + + HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); + List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), + "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); + + // rollback 1st commit and ensure stats reflect the info. + return new BaseRollbackHelper(hoodieTable.getMetaClient(), getConfig()).performRollback(context, + new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"), + rollbackRequests); + } + + private List testUpdateAndRollback(boolean useFileListingMetadata, HoodieWriteConfig writeConfig, SparkRDDWriteClient writeClient) { String newCommitTime = "001"; writeClient.startCommitWithTime(newCommitTime); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 2b4172781c687..43f843a4f3330 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -597,7 +597,7 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String // verify that there is a commit HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build(); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline(); + HoodieTimeline timeline = metaClient.getCommitsTimeline(); if (assertForCommit) { assertEquals(expTotalCommits, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(), @@ -700,6 +700,66 @@ public JavaRDD deleteBatch(SparkRDDWriteClient client, String newCo return result; } + /** + * Insert a batch of records without commit(so that the instant is in-flight). + * + * @param newCommitTime The commit time + * @param numRecords The number of records to insert + */ + @SuppressWarnings("rawtypes, unchecked") + protected void insertBatchWithoutCommit(String newCommitTime, int numRecords) { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoCommit(false) // disable auto commit + .withRollbackUsingMarkers(true) + .build(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + client.startCommitWithTime(newCommitTime); + + List records = dataGen.generateInserts(newCommitTime, numRecords); + JavaRDD writeRecords = jsc.parallelize(records, 1); + + List statuses = client.insert(writeRecords, newCommitTime).collect(); + assertNoWriteErrors(statuses); + } + } + + /** + * Update a batch of records without commit(so that the instant is in-flight). + * + * @param newCommitTime The commit time + * @param baseRecordsToUpdate The base records to update + */ + @SuppressWarnings("rawtypes, unchecked") + protected void updateBatchWithoutCommit(String newCommitTime, List baseRecordsToUpdate) throws IOException { + HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY) + .withAutoCommit(false) // disable auto commit + .withRollbackUsingMarkers(true) + .build(); + + try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) { + client.startCommitWithTime(newCommitTime); + List records = dataGen.generateUpdates(newCommitTime, baseRecordsToUpdate); + JavaRDD writeRecords = jsc.parallelize(records, 1); + client.upsert(writeRecords, newCommitTime); + } + } + + /** + * Asserts the row number in data generator equals to {@code numRows}. + * + * @param numRows The expected row number + */ + protected void assertRowNumberEqualsTo(int numRows) { + // Check the entire dataset has all records still + String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length]; + for (int i = 0; i < fullPartitionPaths.length; i++) { + fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]); + } + assertEquals(numRows, HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(), + "Must contain " + numRows + " records"); + } + /** * Get Cleaner state corresponding to a partition path. * diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java index 609fdb0bd5c0e..842c37449d537 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java @@ -37,6 +37,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.io.storage.HoodieHFileUtils; @@ -96,11 +97,17 @@ public static SparkConf getSparkConfForTest(String appName) { .setMaster("local[4]") .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", "org.apache.spark.HoodieSparkKryoRegistrar") - .set("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension") .set("spark.sql.shuffle.partitions", "4") .set("spark.default.parallelism", "4"); - if (HoodieSparkUtils.gteqSpark3_2()) { + // NOTE: This utility is used in modules where this class might not be present, therefore + // to avoid littering output w/ [[ClassNotFoundException]]s we will skip adding it + // in case this utility is used in the module not providing it + if (canLoadClass("org.apache.spark.sql.hudi.HoodieSparkSessionExtension")) { + sparkConf.set("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension"); + } + + if (canLoadClass("org.apache.spark.sql.hudi.catalog.HoodieCatalog") && HoodieSparkUtils.gteqSpark3_2()) { sparkConf.set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.hudi.catalog.HoodieCatalog"); } @@ -109,6 +116,9 @@ public static SparkConf getSparkConfForTest(String appName) { if (evlogDir != null) { sparkConf.set("spark.eventLog.enabled", "true"); sparkConf.set("spark.eventLog.dir", evlogDir); + sparkConf.set("spark.ui.enabled", "true"); + } else { + sparkConf.set("spark.ui.enabled", "false"); } return SparkRDDReadClient.addHoodieSupport(sparkConf); @@ -326,4 +336,12 @@ private static Option getCommitMetadataForInstant(HoodieTa throw new HoodieException("Failed to read schema from commit metadata", e); } } + + private static boolean canLoadClass(String className) { + try { + return ReflectionUtils.getClass(className) != null; + } catch (Exception e) { + return false; + } + } } 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 8664cf7865d74..c14ad23cab91d 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 @@ -20,10 +20,12 @@ package org.apache.hudi.testutils; 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; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieBaseFile; @@ -41,7 +43,6 @@ import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieIOException; @@ -108,6 +109,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-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala b/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala index c23bbab99b4f0..104d1870eaead 100644 --- a/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala +++ b/hudi-client/hudi-spark-client/src/test/scala/org/apache/spark/sql/TestHoodieUnsafeRowUtils.scala @@ -21,7 +21,7 @@ package org.apache.spark.sql import org.apache.spark.sql.HoodieUnsafeRowUtils.{composeNestedFieldPath, getNestedInternalRowValue, getNestedRowValue} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ -import org.junit.jupiter.api.Assertions.{assertEquals, fail} +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail} import org.junit.jupiter.api.Test class TestHoodieUnsafeRowUtils { @@ -41,11 +41,9 @@ class TestHoodieUnsafeRowUtils { assertEquals( Seq((1, schema(1)), (0, schema(1).dataType.asInstanceOf[StructType](0))), - composeNestedFieldPath(schema, "bar.baz").parts.toSeq) + composeNestedFieldPath(schema, "bar.baz").get.parts.toSeq) - assertThrows(classOf[IllegalArgumentException]) { () => - composeNestedFieldPath(schema, "foo.baz") - } + assertTrue(composeNestedFieldPath(schema, "foo.baz").isEmpty) } @Test @@ -65,36 +63,36 @@ class TestHoodieUnsafeRowUtils { assertEquals( 123, - getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.baz")) + getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.baz").get) ) assertEquals( 456L, - getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.bor")) + getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar.bor").get) ) assertEquals( "str", - getNestedInternalRowValue(row, composeNestedFieldPath(schema, "foo")) + getNestedInternalRowValue(row, composeNestedFieldPath(schema, "foo").get) ) assertEquals( row.getStruct(1, 2), - getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar")) + getNestedInternalRowValue(row, composeNestedFieldPath(schema, "bar").get) ) val rowProperNullable = InternalRow("str", null) assertEquals( null, - getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz")) + getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz").get) ) assertEquals( null, - getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar")) + getNestedInternalRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar").get) ) val rowInvalidNullable = InternalRow(null, InternalRow(123, 456L)) assertThrows(classOf[IllegalArgumentException]) { () => - getNestedInternalRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo")) + getNestedInternalRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo").get) } } @@ -115,36 +113,36 @@ class TestHoodieUnsafeRowUtils { assertEquals( 123, - getNestedRowValue(row, composeNestedFieldPath(schema, "bar.baz")) + getNestedRowValue(row, composeNestedFieldPath(schema, "bar.baz").get) ) assertEquals( 456L, - getNestedRowValue(row, composeNestedFieldPath(schema, "bar.bor")) + getNestedRowValue(row, composeNestedFieldPath(schema, "bar.bor").get) ) assertEquals( "str", - getNestedRowValue(row, composeNestedFieldPath(schema, "foo")) + getNestedRowValue(row, composeNestedFieldPath(schema, "foo").get) ) assertEquals( row.getStruct(1), - getNestedRowValue(row, composeNestedFieldPath(schema, "bar")) + getNestedRowValue(row, composeNestedFieldPath(schema, "bar").get) ) val rowProperNullable = Row("str", null) assertEquals( null, - getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz")) + getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar.baz").get) ) assertEquals( null, - getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar")) + getNestedRowValue(rowProperNullable, composeNestedFieldPath(schema, "bar").get) ) val rowInvalidNullable = Row(null, Row(123, 456L)) assertThrows(classOf[IllegalArgumentException]) { () => - getNestedRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo")) + getNestedRowValue(rowInvalidNullable, composeNestedFieldPath(schema, "foo").get) } } diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml index 0e436f281b9e0..dff34ce2ef53e 100644 --- a/hudi-client/pom.xml +++ b/hudi-client/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 10cbc7fe7d081..766947ea02ced 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java index 395fc100bf18e..24adb1d161ec8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/AvroSchemaUtils.java @@ -18,11 +18,16 @@ package org.apache.hudi.avro; +import org.apache.hudi.exception.SchemaCompatibilityException; + import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; +import org.apache.avro.SchemaCompatibility; +import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -36,12 +41,19 @@ public class AvroSchemaUtils { private AvroSchemaUtils() {} /** - * See {@link #isSchemaCompatible(Schema, Schema, boolean)} doc for more details + * See {@link #isSchemaCompatible(Schema, Schema, boolean, boolean)} doc for more details */ public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema) { return isSchemaCompatible(prevSchema, newSchema, true); } + /** + * See {@link #isSchemaCompatible(Schema, Schema, boolean, boolean)} doc for more details + */ + public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, boolean allowProjection) { + return isSchemaCompatible(prevSchema, newSchema, true, allowProjection); + } + /** * Establishes whether {@code prevSchema} is compatible w/ {@code newSchema}, as * defined by Avro's {@link AvroSchemaCompatibility} @@ -50,15 +62,47 @@ public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema) { * @param newSchema new instance of the schema * @param checkNaming controls whether schemas fully-qualified names should be checked */ - public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, boolean checkNaming) { + public static boolean isSchemaCompatible(Schema prevSchema, Schema newSchema, boolean checkNaming, boolean allowProjection) { // NOTE: We're establishing compatibility of the {@code prevSchema} and {@code newSchema} // as following: {@code newSchema} is considered compatible to {@code prevSchema}, // iff data written using {@code prevSchema} could be read by {@code newSchema} + + // In case schema projection is not allowed, new schema has to have all the same fields as the + // old schema + if (!allowProjection) { + if (!canProject(prevSchema, newSchema)) { + return false; + } + } + AvroSchemaCompatibility.SchemaPairCompatibility result = AvroSchemaCompatibility.checkReaderWriterCompatibility(newSchema, prevSchema, checkNaming); return result.getType() == AvroSchemaCompatibility.SchemaCompatibilityType.COMPATIBLE; } + /** + * Check that each field in the prevSchema can be populated in the newSchema + * @param prevSchema prev schema. + * @param newSchema new schema + * @return true if prev schema is a projection of new schema. + */ + public static boolean canProject(Schema prevSchema, Schema newSchema) { + return canProject(prevSchema, newSchema, Collections.emptySet()); + } + + /** + * Check that each field in the prevSchema can be populated in the newSchema except specified columns + * @param prevSchema prev schema. + * @param newSchema new schema + * @return true if prev schema is a projection of new schema. + */ + public static boolean canProject(Schema prevSchema, Schema newSchema, Set exceptCols) { + return prevSchema.getFields().stream() + .filter(f -> !exceptCols.contains(f.name())) + .map(oldSchemaField -> SchemaCompatibility.lookupWriterField(newSchema, oldSchemaField)) + .noneMatch(Objects::isNull); + } + /** * Generates fully-qualified name for the Avro's schema based on the Table's name * @@ -88,7 +132,7 @@ public static boolean isCompatibleProjectionOf(Schema sourceSchema, Schema targe private static boolean isAtomicSchemasCompatible(Schema oneAtomicType, Schema anotherAtomicType) { // NOTE: Checking for compatibility of atomic types, we should ignore their // corresponding fully-qualified names (as irrelevant) - return isSchemaCompatible(oneAtomicType, anotherAtomicType, false); + return isSchemaCompatible(oneAtomicType, anotherAtomicType, false, true); } /** @@ -253,4 +297,47 @@ public static boolean containsFieldInSchema(Schema schema, String fieldName) { return false; } } + + /** + * Checks whether writer schema is compatible with table schema considering {@code AVRO_SCHEMA_VALIDATE_ENABLE} + * and {@code SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP} options. + * To avoid collision of {@code SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP} and {@code DROP_PARTITION_COLUMNS} + * partition column names should be passed as {@code dropPartitionColNames}. + * Passed empty set means {@code DROP_PARTITION_COLUMNS} is disabled. + * + * @param tableSchema the latest dataset schema + * @param writerSchema writer schema + * @param shouldValidate whether {@link AvroSchemaCompatibility} check being performed + * @param allowProjection whether column dropping check being performed + * @param dropPartitionColNames partition column names to being excluded from column dropping check + * @throws SchemaCompatibilityException if writer schema is not compatible + */ + public static void checkSchemaCompatible( + Schema tableSchema, + Schema writerSchema, + boolean shouldValidate, + boolean allowProjection, + Set dropPartitionColNames) throws SchemaCompatibilityException { + + String errorMessage = null; + + if (!allowProjection && !canProject(tableSchema, writerSchema, dropPartitionColNames)) { + errorMessage = "Column dropping is not allowed"; + } + + // TODO(HUDI-4772) re-enable validations in case partition columns + // being dropped from the data-file after fixing the write schema + if (dropPartitionColNames.isEmpty() && shouldValidate && !isSchemaCompatible(tableSchema, writerSchema)) { + errorMessage = "Failed schema compatibility check"; + } + + if (errorMessage != null) { + String errorDetails = String.format( + "%s\nwriterSchema: %s\ntableSchema: %s", + errorMessage, + writerSchema, + tableSchema); + throw new SchemaCompatibilityException(errorDetails); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/GenericAvroSerializer.java b/hudi-common/src/main/java/org/apache/hudi/avro/GenericAvroSerializer.java new file mode 100644 index 0000000000000..faa36e5694dbd --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/avro/GenericAvroSerializer.java @@ -0,0 +1,146 @@ +/* + * 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.avro; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.nio.ByteBuffer; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericContainer; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.Encoder; +import org.apache.avro.io.EncoderFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; + + +/** + * Custom serializer used for generic Avro containers. + *

+ * Heavily adapted from: + *

+ * GenericAvroSerializer.scala + *

+ * As {@link org.apache.hudi.common.util.SerializationUtils} is not shared between threads and does not concern any + * shuffling operations, compression and decompression cache is omitted as network IO is not a concern. + *

+ * Unlike Spark's implementation, the class and constructor is not initialized with a predefined map of avro schemas. + * This is the case as schemas to read and write are not known beforehand. + * + * @param the subtype of [[GenericContainer]] handled by this serializer + */ +public class GenericAvroSerializer extends Serializer { + + // reuses the same datum reader/writer since the same schema will be used many times + private final HashMap> writerCache = new HashMap<>(); + private final HashMap> readerCache = new HashMap<>(); + + // cache results of Schema to bytes result as the same schema will be used many times + private final HashMap encodeCache = new HashMap<>(); + private final HashMap schemaCache = new HashMap<>(); + + private byte[] getSchemaBytes(Schema schema) { + if (encodeCache.containsKey(schema)) { + return encodeCache.get(schema); + } else { + byte[] schemaBytes = schema.toString().getBytes(StandardCharsets.UTF_8); + encodeCache.put(schema, schemaBytes); + return schemaBytes; + } + } + + private Schema getSchema(byte[] schemaBytes) { + ByteBuffer schemaByteBuffer = ByteBuffer.wrap(schemaBytes); + if (schemaCache.containsKey(schemaByteBuffer)) { + return schemaCache.get(schemaByteBuffer); + } else { + String schema = new String(schemaBytes, StandardCharsets.UTF_8); + Schema parsedSchema = new Schema.Parser().parse(schema); + schemaCache.put(schemaByteBuffer, parsedSchema); + return parsedSchema; + } + } + + private DatumWriter getDatumWriter(Schema schema) { + DatumWriter writer; + if (writerCache.containsKey(schema)) { + writer = writerCache.get(schema); + } else { + writer = new GenericDatumWriter<>(schema); + writerCache.put(schema, writer); + } + return writer; + } + + private DatumReader getDatumReader(Schema schema) { + DatumReader reader; + if (readerCache.containsKey(schema)) { + reader = readerCache.get(schema); + } else { + reader = new GenericDatumReader<>(schema); + readerCache.put(schema, reader); + } + return reader; + } + + private void serializeDatum(D datum, Output output) throws IOException { + Encoder encoder = EncoderFactory.get().directBinaryEncoder(output, null); + Schema schema = datum.getSchema(); + byte[] schemaBytes = getSchemaBytes(schema); + output.writeInt(schemaBytes.length); + output.writeBytes(schemaBytes); + getDatumWriter(schema).write(datum, encoder); + encoder.flush(); + } + + private D deserializeDatum(Input input) throws IOException { + int schemaBytesLen = input.readInt(); + byte[] schemaBytes = input.readBytes(schemaBytesLen); + Schema schema = getSchema(schemaBytes); + Decoder decoder = DecoderFactory.get().directBinaryDecoder(input, null); + return getDatumReader(schema).read(null, decoder); + } + + @Override + public void write(Kryo kryo, Output output, D datum) { + try { + serializeDatum(datum, output); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public D read(Kryo kryo, Input input, Class datumClass) { + try { + return deserializeDatum(input); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java index a4157debdbf2a..1728b99984c5c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java @@ -62,6 +62,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.math.RoundingMode; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.sql.Date; @@ -727,29 +728,24 @@ public static String sanitizeName(String name) { } /** - * Gets record column values into one object. + * Gets record column values into object array. * * @param record Hoodie record. * @param columns Names of the columns to get values. * @param schema {@link Schema} instance. - * @return Column value if a single column, or concatenated String values by comma. + * @return Column value. */ - public static Object getRecordColumnValues(HoodieAvroRecord record, - String[] columns, - Schema schema, boolean consistentLogicalTimestampEnabled) { + public static Object[] getRecordColumnValues(HoodieAvroRecord record, + String[] columns, + Schema schema, + boolean consistentLogicalTimestampEnabled) { try { GenericRecord genericRecord = (GenericRecord) ((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, new Properties()).get()).getData(); - if (columns.length == 1) { - return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true, consistentLogicalTimestampEnabled); - } else { - // TODO this is inefficient, instead we can simply return array of Comparable - StringBuilder sb = new StringBuilder(); - for (String col : columns) { - sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true, consistentLogicalTimestampEnabled)); - } - - return sb.toString(); + List list = new ArrayList<>(); + for (String col : columns) { + list.add(HoodieAvroUtils.getNestedFieldVal(genericRecord, col, true, consistentLogicalTimestampEnabled)); } + return list.toArray(); } catch (IOException e) { throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e); } @@ -993,15 +989,8 @@ private static Object rewritePrimaryTypeWithDiffSchemaType(Object oldValue, Sche || oldSchema.getType() == Schema.Type.LONG || oldSchema.getType() == Schema.Type.FLOAT) { LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) newSchema.getLogicalType(); - BigDecimal bigDecimal = null; - if (oldSchema.getType() == Schema.Type.STRING) { - bigDecimal = new java.math.BigDecimal(oldValue.toString()) - .setScale(decimal.getScale()); - } else { - // Due to Java, there will be precision problems in direct conversion, we should use string instead of use double - bigDecimal = new java.math.BigDecimal(oldValue.toString()) - .setScale(decimal.getScale()); - } + // due to Java, there will be precision problems in direct conversion, we should use string instead of use double + BigDecimal bigDecimal = new java.math.BigDecimal(oldValue.toString()).setScale(decimal.getScale(), RoundingMode.HALF_UP); return DECIMAL_CONVERSION.toFixed(bigDecimal, newSchema, newSchema.getLogicalType()); } } @@ -1053,7 +1042,7 @@ private static Schema getActualSchemaFromUnion(Schema schema, Object data) { } else if (schema.getTypes().size() == 1) { actualSchema = schema.getTypes().get(0); } else { - // deal complex union. this should not happened in hoodie, + // deal complex union. this should not happen in hoodie, // since flink/spark do not write this type. int i = GenericData.get().resolveUnion(schema, data); actualSchema = schema.getTypes().get(i); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigClassProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigClassProperty.java index f0a341eb8244d..9d746365b6930 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigClassProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigClassProperty.java @@ -26,7 +26,7 @@ /** * Annotation for superclasses of {@link HoodieConfig} that includes the * human-readable name of the config class, the config group ({@link ConfigGroups}) - * it belongs to (e.g., spark/ flink/ write) + * it belongs to (e.g., spark/ flink/ write), optional sub-group ({@link ConfigGroups}), * and the description of the config class. */ @Retention(RetentionPolicy.RUNTIME) @@ -36,5 +36,9 @@ ConfigGroups.Names groupName(); + ConfigGroups.SubGroupNames subGroupName() default ConfigGroups.SubGroupNames.NONE; + + boolean areCommonConfigs() default false; + String description(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java index 0488401f10abf..43647da88eb7e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigGroups.java @@ -29,9 +29,11 @@ public class ConfigGroups { * Config group names. */ public enum Names { + ENVIRONMENT_CONFIG("Environment Config"), SPARK_DATASOURCE("Spark Datasource Configs"), FLINK_SQL("Flink Sql Configs"), WRITE_CLIENT("Write Client Configs"), + META_SYNC("Metastore and Catalog Sync Configs"), METRICS("Metrics Configs"), RECORD_PAYLOAD("Record Payload Config"), KAFKA_CONNECT("Kafka Connect Configs"), @@ -44,11 +46,59 @@ public enum Names { } } + public enum SubGroupNames { + INDEX( + "Index Configs", + "Configurations that control indexing behavior, " + + "which tags incoming records as either inserts or updates to older records."), + LOCK( + "Lock Configs", + "Configurations that control locking mechanisms required for concurrency control " + + " between writers to a Hudi table. Concurrency between Hudi's own table services " + + " are auto managed internally."), + COMMIT_CALLBACK( + "Commit Callback Configs", + "Configurations controlling callback behavior into HTTP endpoints, to push " + + "notifications on commits on hudi tables."), + NONE( + "None", + "No subgroup. This description should be hidden."); + + public final String name; + private final String description; + + SubGroupNames(String name, String description) { + this.name = name; + this.description = description; + } + + public String getDescription() { + return description; + } + } + public static String getDescription(Names names) { String description; switch (names) { + case ENVIRONMENT_CONFIG: + description = "Hudi supports passing configurations via a configuration file " + + "`hudi-default.conf` in which each line consists of a key and a value " + + "separated by whitespace or = sign. For example:\n" + + "```\n" + + "hoodie.datasource.hive_sync.mode jdbc\n" + + "hoodie.datasource.hive_sync.jdbcurl jdbc:hive2://localhost:10000\n" + + "hoodie.datasource.hive_sync.support_timestamp false\n" + + "```\n" + + "It helps to have a central configuration file for your common cross " + + "job configurations/tunings, so all the jobs on your cluster can utilize it. " + + "It also works with Spark SQL DML/DDL, and helps avoid having to pass configs " + + "inside the SQL statements.\n\n" + + "By default, Hudi would load the configuration file under `/etc/hudi/conf` " + + "directory. You can specify a different configuration directory location by " + + "setting the `HUDI_CONF_DIR` environment variable."; + break; case SPARK_DATASOURCE: - description = "These configs control the Hudi Spark Datasource, " + description = "These configs control the Hudi Spark Datasource, " + "providing ability to define keys/partitioning, pick out the write operation, " + "specify how to merge records or choosing query type to read."; break; @@ -65,8 +115,11 @@ public static String getDescription(Names names) { + "write schema, cleaning etc. Although Hudi provides sane defaults, from time-time " + "these configs may need to be tweaked to optimize for specific workloads."; break; + case META_SYNC: + description = "Configurations used by the Hudi to sync metadata to external metastores and catalogs."; + break; case RECORD_PAYLOAD: - description = "This is the lowest level of customization offered by Hudi. " + description = "This is the lowest level of customization offered by Hudi. " + "Record payloads define how to produce new values to upsert based on incoming " + "new record and stored old record. Hudi provides default implementations such as " + "OverwriteWithLatestAvroPayload which simply update table with the latest/last-written record. " @@ -80,6 +133,9 @@ public static String getDescription(Names names) { case KAFKA_CONNECT: description = "These set of configs are used for Kafka Connect Sink Connector for writing Hudi Tables"; break; + case AWS: + description = "Configurations specific to Amazon Web Services."; + break; default: description = "Please fill in the description for Config Group Name: " + names.name; break; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java index 08f36512c9150..364fc4203a817 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java @@ -34,7 +34,7 @@ /** * ConfigProperty describes a configuration property. It contains the configuration * key, deprecated older versions of the key, and an optional default value for the configuration, - * configuration descriptions and also the an infer mechanism to infer the configuration value + * configuration descriptions and also an inferring mechanism to infer the configuration value * based on other configurations. * * @param The type of the default value. @@ -45,6 +45,8 @@ public class ConfigProperty implements Serializable { private final T defaultValue; + private final String docOnDefaultValue; + private final String doc; private final Option sinceVersion; @@ -58,10 +60,13 @@ public class ConfigProperty implements Serializable { // provide the ability to infer config value based on other configs private final Option>> inferFunction; - ConfigProperty(String key, T defaultValue, String doc, Option sinceVersion, - Option deprecatedVersion, Option>> inferFunc, Set validValues, String... alternatives) { + ConfigProperty(String key, T defaultValue, String docOnDefaultValue, String doc, + Option sinceVersion, Option deprecatedVersion, + Option>> inferFunc, Set validValues, + String... alternatives) { this.key = Objects.requireNonNull(key); this.defaultValue = defaultValue; + this.docOnDefaultValue = docOnDefaultValue; this.doc = doc; this.sinceVersion = sinceVersion; this.deprecatedVersion = deprecatedVersion; @@ -85,6 +90,11 @@ public boolean hasDefaultValue() { return defaultValue != null; } + public String getDocOnDefaultValue() { + return StringUtils.isNullOrEmpty(docOnDefaultValue) + ? StringUtils.EMPTY_STRING : docOnDefaultValue; + } + public String doc() { return StringUtils.isNullOrEmpty(doc) ? StringUtils.EMPTY_STRING : doc; } @@ -97,7 +107,11 @@ public Option getDeprecatedVersion() { return deprecatedVersion; } - Option>> getInferFunc() { + public boolean hasInferFunction() { + return getInferFunction().isPresent(); + } + + Option>> getInferFunction() { return inferFunction; } @@ -115,32 +129,32 @@ public List getAlternatives() { public ConfigProperty withDocumentation(String doc) { Objects.requireNonNull(doc); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); } public ConfigProperty withValidValues(String... validValues) { Objects.requireNonNull(validValues); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, new HashSet<>(Arrays.asList(validValues)), alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, new HashSet<>(Arrays.asList(validValues)), alternatives); } public ConfigProperty withAlternatives(String... alternatives) { Objects.requireNonNull(alternatives); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, validValues, alternatives); } public ConfigProperty sinceVersion(String sinceVersion) { Objects.requireNonNull(sinceVersion); - return new ConfigProperty<>(key, defaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, validValues, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, validValues, alternatives); } public ConfigProperty deprecatedAfter(String deprecatedVersion) { Objects.requireNonNull(deprecatedVersion); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, validValues, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, validValues, alternatives); } public ConfigProperty withInferFunction(Function> inferFunction) { Objects.requireNonNull(inferFunction); - return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), validValues, alternatives); + return new ConfigProperty<>(key, defaultValue, docOnDefaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), validValues, alternatives); } /** @@ -174,13 +188,22 @@ public static final class PropertyBuilder { } public ConfigProperty defaultValue(T value) { + return defaultValue(value, ""); + } + + public ConfigProperty defaultValue(T value, String docOnDefaultValue) { Objects.requireNonNull(value); - ConfigProperty configProperty = new ConfigProperty<>(key, value, "", Option.empty(), Option.empty(), Option.empty(), Collections.emptySet()); + Objects.requireNonNull(docOnDefaultValue); + ConfigProperty configProperty = new ConfigProperty<>(key, value, docOnDefaultValue, "", Option.empty(), Option.empty(), Option.empty(), Collections.emptySet()); return configProperty; } public ConfigProperty noDefaultValue() { - ConfigProperty configProperty = new ConfigProperty<>(key, null, "", Option.empty(), + return noDefaultValue(""); + } + + public ConfigProperty noDefaultValue(String docOnDefaultValue) { + ConfigProperty configProperty = new ConfigProperty<>(key, null, docOnDefaultValue, "", Option.empty(), Option.empty(), Option.empty(), Collections.emptySet()); return configProperty; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java index 25ed017fee86c..e8fa0b6255884 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java @@ -67,6 +67,15 @@ public class HoodieCommonConfig extends HoodieConfig { .defaultValue(true) .withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map"); + public static final ConfigProperty HOODIE_FS_ATOMIC_CREATION_SUPPORT = ConfigProperty + .key("hoodie.fs.atomic_creation.support") + .defaultValue("") + .withDocumentation("This config is used to specify the file system which supports atomic file creation . " + + "atomic means that an operation either succeeds and has an effect or has fails and has no effect;" + + " now this feature is used by FileSystemLockProvider to guaranteeing that only one writer can create the lock file at a time." + + " since some FS does not support atomic file creation (eg: S3), we decide the FileSystemLockProvider only support HDFS,local FS" + + " and View FS as default. if you want to use FileSystemLockProvider with other FS, you can set this config with the FS scheme, eg: fs1,fs2"); + public ExternalSpillableMap.DiskMapType getSpillableDiskMapType() { return ExternalSpillableMap.DiskMapType.valueOf(getString(SPILLABLE_DISK_MAP_TYPE).toUpperCase(Locale.ROOT)); } 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 a48e4202bf962..b2b8ba34f8760 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 @@ -51,6 +51,10 @@ public HoodieConfig(Properties props) { this.props = new TypedProperties(props); } + public HoodieConfig(TypedProperties props) { + this.props = props; + } + public void setValue(ConfigProperty cfg, String val) { cfg.checkValues(val); props.setProperty(cfg.key(), val); @@ -64,11 +68,22 @@ public void setAll(Properties properties) { props.putAll(properties); } + /** + * Sets the default value of a config if user does not set it already. + * The default value can only be set if the config property has a built-in + * default value or an infer function. When the infer function is present, + * the infer function is used first to derive the config value based on other + * configs. If the config value cannot be inferred, the built-in default value + * is used if present. + * + * @param configProperty Config to set a default value. + * @param Data type of the config. + */ public void setDefaultValue(ConfigProperty configProperty) { if (!contains(configProperty)) { Option inferValue = Option.empty(); - if (configProperty.getInferFunc().isPresent()) { - inferValue = configProperty.getInferFunc().get().apply(this); + if (configProperty.hasInferFunction()) { + inferValue = configProperty.getInferFunction().get().apply(this); } if (inferValue.isPresent() || configProperty.hasDefaultValue()) { props.setProperty( @@ -120,7 +135,7 @@ protected void setDefaults(String configClassName) { .forEach(f -> { try { ConfigProperty cfgProp = (ConfigProperty) f.get("null"); - if (cfgProp.hasDefaultValue() || cfgProp.getInferFunc().isPresent()) { + if (cfgProp.hasDefaultValue() || cfgProp.hasInferFunction()) { setDefaultValue(cfgProp); } } catch (IllegalAccessException e) { @@ -142,14 +157,6 @@ public List getSplitStrings(ConfigProperty configProperty, String return StringUtils.split(getString(configProperty), delimiter); } - public List getSplitStringsOrDefault(ConfigProperty configProperty) { - return getSplitStringsOrDefault(configProperty, ","); - } - - public List getSplitStringsOrDefault(ConfigProperty configProperty, String delimiter) { - return StringUtils.split(getStringOrDefault(configProperty), delimiter); - } - public String getString(String key) { return props.getProperty(key); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 81f2c1daeff3d..9b84466090db8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -42,6 +42,7 @@ public final class HoodieMetadataConfig extends HoodieConfig { public static final String METADATA_PREFIX = "hoodie.metadata"; + public static final String OPTIMIZED_LOG_BLOCKS_SCAN = ".optimized.log.blocks.scan.enable"; // Enable the internal Metadata Table which saves file listings public static final ConfigProperty ENABLE = ConfigProperty @@ -237,12 +238,12 @@ public final class HoodieMetadataConfig extends HoodieConfig { + "metadata table which are never added before. This config determines how to handle " + "such spurious deletes"); - public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = ConfigProperty - .key(METADATA_PREFIX + ".log.record.reader.use.scanV2") + public static final ConfigProperty ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN = ConfigProperty + .key(METADATA_PREFIX + OPTIMIZED_LOG_BLOCKS_SCAN) .defaultValue(false) .sinceVersion("0.13.0") - .withDocumentation("ScanV2 logic address all the multiwriter challenges while appending to log files. " - + "It also differentiates original blocks written by ingestion writers and compacted blocks written log compaction."); + .withDocumentation("Optimized log blocks scanner that addresses all the multiwriter use-cases while appending to log files. " + + "It also differentiates original blocks written by ingestion writers and compacted blocks written by log compaction."); private HoodieMetadataConfig() { super(); @@ -328,8 +329,8 @@ public boolean ignoreSpuriousDeletes() { return getBoolean(IGNORE_SPURIOUS_DELETES); } - public boolean getUseLogRecordReaderScanV2() { - return getBoolean(USE_LOG_RECORD_READER_SCAN_V2); + public boolean doEnableOptimizedLogBlocksScan() { + return getBoolean(ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN); } /** @@ -478,8 +479,8 @@ public Builder withProperties(Properties properties) { return this; } - public Builder withLogRecordReaderScanV2(boolean useLogRecordReaderScanV2) { - metadataConfig.setValue(USE_LOG_RECORD_READER_SCAN_V2, String.valueOf(useLogRecordReaderScanV2)); + public Builder withOptimizedLogBlocksScan(boolean enableOptimizedLogBlocksScan) { + metadataConfig.setValue(ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN, String.valueOf(enableOptimizedLogBlocksScan)); return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java index 89841ed2fafa1..e650c8dac5c38 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java @@ -312,7 +312,7 @@ public Builder hfileBlockSize(int blockSize) { return this; } - public Builder logFileDataBlockMaxSize(int dataBlockSize) { + public Builder logFileDataBlockMaxSize(long dataBlockSize) { storageConfig.setValue(LOGFILE_DATA_BLOCK_MAX_SIZE, String.valueOf(dataBlockSize)); return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableServiceManagerConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableServiceManagerConfig.java index 501b7fc421b81..f40ba8be35440 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableServiceManagerConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTableServiceManagerConfig.java @@ -26,11 +26,10 @@ /** * Configurations used by the Hudi Table Service Manager. + * + * TODO: enable docs gen by adding {@link ConfigClassProperty} after TSM is landed (HUDI-3475) */ @Immutable -@ConfigClassProperty(name = "Table Service Manager Configs", - groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Configurations used by the Hudi Table Service Manager.") public class HoodieTableServiceManagerConfig extends HoodieConfig { public static final String TABLE_SERVICE_MANAGER_PREFIX = "hoodie.table.service.manager"; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java index f246b75be7aeb..3db8210cadee9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.Enumeration; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Properties; import java.util.stream.Collectors; @@ -119,4 +120,43 @@ public double getDouble(String property) { public double getDouble(String property, double defaultValue) { return containsKey(property) ? Double.parseDouble(getProperty(property)) : defaultValue; } + + /** + * This method is introduced to get rid of the scala compile error: + *
+   *   
+   *   ambiguous reference to overloaded definition,
+   *   both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit
+   *   and  method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit
+   *   match argument types (java.util.HashMap[Nothing,Nothing])
+   *       properties.putAll(new java.util.HashMap())
+   *   
+   * 
+ * + * @param items The new items to put + */ + public static TypedProperties fromMap(Map items) { + TypedProperties props = new TypedProperties(); + props.putAll(items); + return props; + } + + /** + * This method is introduced to get rid of the scala compile error: + *
+   *   
+   *   ambiguous reference to overloaded definition,
+   *   both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit
+   *   and  method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit
+   *   match argument types (java.util.HashMap[Nothing,Nothing])
+   *       properties.putAll(new java.util.HashMap())
+   *   
+   * 
+ * + * @param props The properties + * @param items The new items to put + */ + public static void putAll(TypedProperties props, Map items) { + props.putAll(items); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java index 1d56e63fad928..9ce0947883f6b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java @@ -105,9 +105,9 @@ HoodieData mapPartitions(SerializableFunction, Iterator> func, boolean preservesPartitioning); /** - * Maps every element in the collection into a collection of the new elements (provided by - * {@link Iterator}) using provided mapping {@code func}, subsequently flattening the result - * (by concatenating) into a single collection + * Maps every element in the collection into a collection of the new elements using provided + * mapping {@code func}, subsequently flattening the result (by concatenating) into a single + * collection * * This is an intermediate operation * @@ -117,6 +117,17 @@ HoodieData mapPartitions(SerializableFunction, */ HoodieData flatMap(SerializableFunction> func); + /** + * Maps every element in the collection into a collection of the {@link Pair}s of new elements + * using provided mapping {@code func}, subsequently flattening the result (by concatenating) into + * a single collection + * + * NOTE: That this operation will convert container from {@link HoodieData} to {@link HoodiePairData} + * + * This is an intermediate operation + */ + HoodiePairData flatMapToPair(SerializableFunction>> func); + /** * Maps every element in the collection using provided mapping {@code func} into a {@link Pair} * of elements {@code K} and {@code V} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java index b2a503a85b323..c6287a744e0e1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java @@ -125,6 +125,16 @@ public HoodieData flatMap(SerializableFunction> func) { return new HoodieListData<>(mappedStream, lazy); } + @Override + public HoodiePairData flatMapToPair(SerializableFunction>> func) { + Function>> mapper = throwingMapWrapper(func); + Stream> mappedStream = asStream().flatMap(e -> + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(mapper.apply(e), Spliterator.ORDERED), true)); + + return new HoodieListPairData<>(mappedStream, lazy); + } + @Override public HoodiePairData mapToPair(SerializablePairFunction func) { Function> throwableMapToPairFunc = throwingMapToPairWrapper(func); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListPairData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListPairData.java index a389649548e98..39ce141157593 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListPairData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListPairData.java @@ -23,15 +23,20 @@ import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.MappingIterator; import org.apache.hudi.common.util.collection.Pair; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Spliterator; +import java.util.Spliterators; import java.util.function.Function; import java.util.stream.Collector; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper; import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper; @@ -136,6 +141,24 @@ public HoodieData map(SerializableFunction, O> func) { return new HoodieListData<>(asStream().map(uncheckedMapper), lazy); } + @Override + public HoodiePairData mapValues(SerializableFunction func) { + Function uncheckedMapper = throwingMapWrapper(func); + return new HoodieListPairData<>(asStream().map(p -> Pair.of(p.getKey(), uncheckedMapper.apply(p.getValue()))), lazy); + } + + public HoodiePairData flatMapValues(SerializableFunction> func) { + Function> uncheckedMapper = throwingMapWrapper(func); + return new HoodieListPairData<>(asStream().flatMap(p -> { + Iterator mappedValuesIterator = uncheckedMapper.apply(p.getValue()); + Iterator> mappedPairsIterator = + new MappingIterator<>(mappedValuesIterator, w -> Pair.of(p.getKey(), w)); + + return StreamSupport.stream( + Spliterators.spliteratorUnknownSize(mappedPairsIterator, Spliterator.ORDERED), true); + }), lazy); + } + @Override public HoodiePairData mapToPair(SerializablePairFunction, L, W> mapToPairFunc) { return new HoodieListPairData<>(asStream().map(p -> throwingMapToPairWrapper(mapToPairFunc).apply(p)), lazy); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java index 49fa7174da9a6..1d3622786fd07 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java @@ -90,12 +90,17 @@ public interface HoodiePairData extends Serializable { HoodiePairData reduceByKey(SerializableBiFunction combiner, int parallelism); /** - * @param func serializable map function. - * @param output object type. - * @return {@link HoodieData} containing the result. Actual execution may be deferred. + * Maps key-value pairs of this {@link HoodiePairData} container leveraging provided mapper + * + * NOTE: That this returns {@link HoodieData} and not {@link HoodiePairData} */ HoodieData map(SerializableFunction, O> func); + /** + * Maps values of this {@link HoodiePairData} container leveraging provided mapper + */ + HoodiePairData mapValues(SerializableFunction func); + /** * @param mapToPairFunc serializable map function to generate another pair. * @param new key type. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuard.java index 15d83c4bc4542..cd649a6828765 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuard.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuard.java @@ -69,7 +69,7 @@ enum FileVisibility { * * @param dirPath Directory Path * @param files Files - * @param targetVisibility Target Visibitlity + * @param targetVisibility Target Visibility * @throws IOException * @throws TimeoutException */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java index 323e41f074efe..e0ae2fd5d3cf7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java @@ -137,8 +137,8 @@ public Builder withConsistencyCheckEnabled(boolean enabled) { return this; } - public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) { - consistencyGuardConfig.setValue(INITIAL_CHECK_INTERVAL_MS, String.valueOf(initialIntevalMs)); + public Builder withInitialConsistencyCheckIntervalMs(int initialIntervalMs) { + consistencyGuardConfig.setValue(INITIAL_CHECK_INTERVAL_MS, String.valueOf(initialIntervalMs)); return this; } 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 bb9a8d7159d46..a17ae25250b79 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 @@ -33,6 +33,7 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.exception.InvalidHoodiePathException; import org.apache.hudi.hadoop.CachingPath; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -81,6 +82,7 @@ public class FSUtils { // Archive log files are of this pattern - .commits_.archive.1_1-0-1 public static final Pattern LOG_FILE_PATTERN = Pattern.compile("^\\.(.+)_(.*)\\.(log|archive)\\.(\\d+)(_((\\d+)-(\\d+)-(\\d+))(.cdc)?)?"); + public static final Pattern PREFIX_BY_FILE_ID_PATTERN = Pattern.compile("^(.+)-(\\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; @@ -225,7 +227,8 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP String fullPartitionPathStr = fullPartitionPath.toString(); if (!fullPartitionPathStr.startsWith(basePath.toString())) { - throw new IllegalArgumentException("Partition path does not belong to base-path"); + throw new IllegalArgumentException("Partition path \"" + fullPartitionPathStr + + "\" does not belong to base-path \"" + basePath + "\""); } int partitionStartIndex = fullPartitionPathStr.indexOf(basePath.getName(), @@ -350,6 +353,17 @@ public static String createNewFileIdPfx() { return UUID.randomUUID().toString(); } + /** + * Returns prefix for a file group from fileId. + */ + public static String getFileIdPfxFromFileId(String fileId) { + Matcher matcher = PREFIX_BY_FILE_ID_PATTERN.matcher(fileId); + if (!matcher.find()) { + throw new HoodieValidationException("Failed to get prefix from " + fileId); + } + return matcher.group(1); + } + public static String createNewFileId(String idPfx, int id) { return String.format("%s-%d", idPfx, id); } @@ -582,7 +596,7 @@ public static Short getDefaultReplication(FileSystem fs, Path path) { /** * When a file was opened and the task died without closing the stream, another task executor cannot open because the * existing lease will be active. We will try to recover the lease, from HDFS. If a data node went down, it takes - * about 10 minutes for the lease to be rocovered. But if the client dies, this should be instant. + * about 10 minutes for the lease to be recovered. But if the client dies, this should be instant. */ public static boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p) throws IOException, InterruptedException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java index 2247b92a4cee0..f73221f96ea3f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FailSafeConsistencyGuard.java @@ -139,7 +139,7 @@ private void waitForFileVisibility(Path filePath, FileVisibility visibility) thr } /** - * Retries the predicate for condfigurable number of times till we the predicate returns success. + * Retries the predicate for configurable number of times till we the predicate returns success. * * @param dir directory of interest in which list of files are checked for visibility * @param files List of files to check for visibility 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 d10813021d402..68bbe0a0bc426 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 @@ -270,4 +270,15 @@ public Configuration getConf() { public String getScheme() { return fileSystem.getScheme(); } + + @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/fs/OptimisticConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java index ded7ccce90ebe..d3d5c44883fe3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/OptimisticConsistencyGuard.java @@ -41,7 +41,7 @@ * Step1 and Step2 is handled by {@link FailSafeConsistencyGuard}. * * We are simplifying these steps with {@link OptimisticConsistencyGuard}. - * Step1: Check if all files adhere to visibility event. If yes, proceed to Sptep 3. + * Step1: Check if all files adhere to visibility event. If yes, proceed to Step 3. * Step2: If not, Sleep for a configured threshold and then proceed to next step. * Step3: issue deletes. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareDataOutputStream.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareDataOutputStream.java index ba4fe66f82ba1..1cc3da6fe3cb3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareDataOutputStream.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/SizeAwareDataOutputStream.java @@ -25,11 +25,11 @@ import java.util.concurrent.atomic.AtomicLong; /** - * Wrapper for DataOutpuStream to keep track of number of bytes written. + * Wrapper for DataOutputStream to keep track of number of bytes written. */ public class SizeAwareDataOutputStream { - // Actual outpuStream + // Actual outputStream private DataOutputStream outputStream; // Counter to keep track of number of bytes written private AtomicLong size; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index cb8b4c511818f..24f1b91bd41ab 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -25,65 +25,69 @@ */ public enum StorageSchemes { // Local filesystem - FILE("file", false, false), + FILE("file", false, false, true), // Hadoop File System - HDFS("hdfs", true, false), + HDFS("hdfs", true, false, true), // Baidu Advanced File System - AFS("afs", true, null), + AFS("afs", true, null, null), // Mapr File System - MAPRFS("maprfs", true, null), + MAPRFS("maprfs", true, null, null), // Apache Ignite FS - IGNITE("igfs", true, null), + IGNITE("igfs", true, null, null), // AWS S3 - S3A("s3a", false, true), S3("s3", false, true), + S3A("s3a", false, true, null), S3("s3", false, true, null), // Google Cloud Storage - GCS("gs", false, true), + GCS("gs", false, true, null), // Azure WASB - WASB("wasb", false, null), WASBS("wasbs", false, null), + WASB("wasb", false, null, null), WASBS("wasbs", false, null, null), // Azure ADLS - ADL("adl", false, null), + ADL("adl", false, null, null), // Azure ADLS Gen2 - ABFS("abfs", false, null), ABFSS("abfss", false, null), + ABFS("abfs", false, null, null), ABFSS("abfss", false, null, null), // Aliyun OSS - OSS("oss", false, null), + OSS("oss", false, null, null), // View FS for federated setups. If federating across cloud stores, then append support is false - VIEWFS("viewfs", true, null), + // View FS support atomic creation + VIEWFS("viewfs", true, null, true), //ALLUXIO - ALLUXIO("alluxio", false, null), + ALLUXIO("alluxio", false, null, null), // Tencent Cloud Object Storage - COSN("cosn", false, null), + COSN("cosn", false, null, null), // Tencent Cloud HDFS - CHDFS("ofs", true, null), + CHDFS("ofs", true, null, null), // Tencent Cloud CacheFileSystem - GOOSEFS("gfs", false, null), + GOOSEFS("gfs", false, null, null), // Databricks file system - DBFS("dbfs", false, null), + DBFS("dbfs", false, null, null), // IBM Cloud Object Storage - COS("cos", false, null), + COS("cos", false, null, null), // Huawei Cloud Object Storage - OBS("obs", false, null), + OBS("obs", false, null, null), // Kingsoft Standard Storage ks3 - KS3("ks3", false, null), + KS3("ks3", false, null, null), // JuiceFileSystem - JFS("jfs", true, null), + JFS("jfs", true, null, null), // Baidu Object Storage - BOS("bos", false, null), + BOS("bos", false, null, null), // Oracle Cloud Infrastructure Object Storage - OCI("oci", false, null), + OCI("oci", false, null, null), // Volcengine Object Storage - TOS("tos", false, null), + TOS("tos", false, null, null), // Volcengine Cloud HDFS - CFS("cfs", true, null); + CFS("cfs", true, null, null); private String scheme; private boolean supportsAppend; // null for uncertain if write is transactional, please update this for each FS private Boolean isWriteTransactional; + // null for uncertain if dfs support atomic create&delete, please update this for each FS + private Boolean supportAtomicCreation; - StorageSchemes(String scheme, boolean supportsAppend, Boolean isWriteTransactional) { + StorageSchemes(String scheme, boolean supportsAppend, Boolean isWriteTransactional, Boolean supportAtomicCreation) { this.scheme = scheme; this.supportsAppend = supportsAppend; this.isWriteTransactional = isWriteTransactional; + this.supportAtomicCreation = supportAtomicCreation; } public String getScheme() { @@ -98,6 +102,10 @@ public boolean isWriteTransactional() { return isWriteTransactional != null && isWriteTransactional; } + public boolean isAtomicCreationSupported() { + return supportAtomicCreation != null && supportAtomicCreation; + } + public static boolean isSchemeSupported(String scheme) { return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme)); } @@ -116,4 +124,11 @@ public static boolean isWriteTransactional(String scheme) { return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.isWriteTransactional() && s.scheme.equals(scheme)); } + + public static boolean isAtomicCreationSupported(String scheme) { + if (!isSchemeSupported(scheme)) { + throw new IllegalArgumentException("Unsupported scheme :" + scheme); + } + return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.isAtomicCreationSupported() && s.scheme.equals(scheme)); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/function/ThrowingConsumer.java b/hudi-common/src/main/java/org/apache/hudi/common/function/ThrowingConsumer.java new file mode 100644 index 0000000000000..1cbae113fcbce --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/function/ThrowingConsumer.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.function; + +/** + * Throwing counterpart of {@link java.util.function.Consumer} + */ +@FunctionalInterface +public interface ThrowingConsumer { + + void accept(T t) throws Exception; + +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java index 451008257a535..f22cfc083340a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java @@ -62,7 +62,7 @@ private Option handleDeleteOperation(IndexedRecord insertValue) t boolean delete = false; if (insertValue instanceof GenericRecord) { GenericRecord record = (GenericRecord) insertValue; - delete = record.get(OP_FIELD) != null && record.get(OP_FIELD).toString().equalsIgnoreCase("D"); + delete = isDMSDeleteRecord(record); } return delete ? Option.empty() : Option.of(insertValue); @@ -94,4 +94,13 @@ public Option combineAndGetUpdateValue(IndexedRecord currentValue } return handleDeleteOperation(insertValue.get()); } + + @Override + protected boolean isDeleteRecord(GenericRecord record) { + return isDMSDeleteRecord(record) || super.isDeleteRecord(record); + } + + private static boolean isDMSDeleteRecord(GenericRecord record) { + return record.get(OP_FIELD) != null && record.get(OP_FIELD).toString().equalsIgnoreCase("D"); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java index aaafe61abff9a..85e46690287d3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/BaseAvroPayload.java @@ -83,7 +83,7 @@ public boolean canProduceSentinel() { * @param genericRecord instance of {@link GenericRecord} of interest. * @returns {@code true} if record represents a delete record. {@code false} otherwise. */ - protected static boolean isDeleteRecord(GenericRecord genericRecord) { + protected boolean isDeleteRecord(GenericRecord genericRecord) { final String isDeleteKey = HoodieRecord.HOODIE_IS_DELETED_FIELD; // Modify to be compatible with new version Avro. // The new version Avro throws for GenericRecord.get if the field name diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java index 003b591c20c05..296e95e8bfa6a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java @@ -28,6 +28,15 @@ * we need to keep the ordering val to combine with the data records when merging, or the data loss * may occur if there are intermediate deletions for the inputs * (a new INSERT comes after a DELETE in one input batch). + * + * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING + * + * This class is serialized (using Kryo) as part of {@code HoodieDeleteBlock} to make + * sure this stays backwards-compatible we can't MAKE ANY CHANGES TO THIS CLASS (add, + * delete, reorder or change types of the fields in this class, make class final, etc) + * as this would break its compatibility with already persisted blocks. + * + * Check out HUDI-5760 for more details */ public class DeleteRecord implements Serializable { private static final long serialVersionUID = 1L; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java index 7d497408e6467..db77c21e8f465 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java @@ -112,28 +112,18 @@ public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { } @Override - public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { - GenericRecord record = HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema); - return new HoodieAvroIndexedRecord(key, record, operation, metaData); + public HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props) { + GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(data, targetSchema); + updateMetadataValuesInternal(newAvroRecord, metadataValues); + return new HoodieAvroIndexedRecord(key, newAvroRecord, operation, metaData); } @Override - public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) { GenericRecord record = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols); return new HoodieAvroIndexedRecord(key, record, operation, metaData); } - @Override - public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { - metadataValues.getKv().forEach((key, value) -> { - if (value != null) { - ((GenericRecord) data).put(key, value); - } - }); - - return new HoodieAvroIndexedRecord(key, data, operation, metaData); - } - @Override public HoodieRecord truncateRecordKey(Schema recordSchema, Properties props, String keyFieldName) { ((GenericRecord) data).put(keyFieldName, StringUtils.EMPTY_STRING); @@ -234,4 +224,18 @@ protected final IndexedRecord readRecordPayload(Kryo kryo, Input input) { return kryo.readObjectOrNull(input, GenericRecord.class, avroSerializer); } + + static void updateMetadataValuesInternal(GenericRecord avroRecord, MetadataValues metadataValues) { + if (metadataValues.isEmpty()) { + return; // no-op + } + + String[] values = metadataValues.getValues(); + for (int pos = 0; pos < values.length; ++pos) { + String value = values[pos]; + if (value != null) { + avroRecord.put(HoodieMetadataField.values()[pos].getFieldName(), value); + } + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java index 62001356b4fad..b3f2aa4a6fb61 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.avro.Schema; @@ -37,6 +38,8 @@ import java.util.Map; import java.util.Properties; +import static org.apache.hudi.common.model.HoodieAvroIndexedRecord.updateMetadataValuesInternal; + /** * Implementation of {@link HoodieRecord} using Avro payload. * @@ -114,7 +117,7 @@ public HoodieRecordType getRecordType() { @Override public Object[] getColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) { - return new Object[]{HoodieAvroUtils.getRecordColumnValues(this, columns, recordSchema, consistentLogicalTimestampEnabled)}; + return HoodieAvroUtils.getRecordColumnValues(this, columns, recordSchema, consistentLogicalTimestampEnabled); } @Override @@ -123,31 +126,26 @@ public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { } @Override - public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { - Option avroRecordPayloadOpt = getData().getInsertValue(recordSchema, props); - GenericRecord avroPayloadInNewSchema = - HoodieAvroUtils.rewriteRecord((GenericRecord) avroRecordPayloadOpt.get(), targetSchema); - return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroPayloadInNewSchema), getOperation(), this.currentLocation, this.newLocation); - } - - @Override - public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { - GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); - GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); - return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation(), this.currentLocation, this.newLocation); + public HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props) { + try { + Option avroRecordOpt = getData().getInsertValue(recordSchema, props); + GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecordOpt.get(), targetSchema); + updateMetadataValuesInternal(newAvroRecord, metadataValues); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(newAvroRecord), getOperation(), this.currentLocation, this.newLocation); + } catch (IOException e) { + throw new HoodieIOException("Failed to deserialize record!", e); + } } @Override - public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { - GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); - - metadataValues.getKv().forEach((key, value) -> { - if (value != null) { - avroRecordPayload.put(key, value); - } - }); - - return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation(), this.currentLocation, this.newLocation); + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) { + try { + GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, props).get(); + GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols); + return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation(), this.currentLocation, this.newLocation); + } catch (IOException e) { + throw new HoodieIOException("Failed to deserialize record!", e); + } } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java index e49d560b74c6f..252e11135afcf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java @@ -27,7 +27,6 @@ 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.metadata.HoodieMetadataPayload; import java.io.IOException; import java.util.Properties; @@ -50,7 +49,7 @@ public Option> merge(HoodieRecord older, Schema oldSc switch (legacyOperatingMode) { case PRE_COMBINING: - HoodieRecord res = preCombine(older, newer); + HoodieRecord res = preCombine(older, newer, newSchema, props); if (res == older) { return Option.of(Pair.of(res, oldSchema)); } else { @@ -71,13 +70,9 @@ public HoodieRecordType getRecordType() { return HoodieRecordType.AVRO; } - private HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer) { - HoodieRecordPayload picked = unsafeCast(((HoodieAvroRecord) newer).getData().preCombine(((HoodieAvroRecord) older).getData())); - if (picked instanceof HoodieMetadataPayload) { - // NOTE: HoodieMetadataPayload return a new payload - return new HoodieAvroRecord(newer.getKey(), picked, newer.getOperation()); - } - return picked.equals(((HoodieAvroRecord) newer).getData()) ? newer : older; + private HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) { + HoodieRecordPayload payload = unsafeCast(((HoodieAvroRecord) newer).getData().preCombine(((HoodieAvroRecord) older).getData(), schema, props)); + return new HoodieAvroRecord(newer.getKey(), payload, newer.getOperation()); } private Option combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java index e0f51fcbaa37c..0185d22d949bd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieConsistentHashingMetadata.java @@ -49,6 +49,7 @@ public class HoodieConsistentHashingMetadata implements Serializable { */ public static final int HASH_VALUE_MASK = Integer.MAX_VALUE; public static final String HASHING_METADATA_FILE_SUFFIX = ".hashing_meta"; + public static final String HASHING_METADATA_COMMIT_FILE_SUFFIX = ".commit"; private final short version; private final String partitionPath; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java index e2f9334f8a2ec..3c7927b6b65f4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java @@ -99,17 +99,12 @@ public HoodieRecord joinWith(HoodieRecord other, Schema targetSchema) { } @Override - public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException { + public HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props) { throw new UnsupportedOperationException(); } @Override - public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException { + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) { throw new UnsupportedOperationException(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java index 5208cece1cb56..436758f96f404 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java @@ -18,11 +18,6 @@ package org.apache.hudi.common.model; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.KryoSerializable; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; - import java.io.Serializable; import java.util.Objects; @@ -31,8 +26,17 @@ *

* - recordKey : a recordKey that acts as primary key for a record. * - partitionPath : the partition path of a record. + * + * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING + * + * This class is serialized (using Kryo) as part of {@code HoodieDeleteBlock} to make + * sure this stays backwards-compatible we can't MAKE ANY CHANGES TO THIS CLASS (add, + * delete, reorder or change types of the fields in this class, make class final, etc) + * as this would break its compatibility with already persisted blocks. + * + * Check out HUDI-5760 for more details */ -public final class HoodieKey implements Serializable, KryoSerializable { +public class HoodieKey implements Serializable { private String recordKey; private String partitionPath; @@ -86,16 +90,4 @@ public String toString() { sb.append('}'); return sb.toString(); } - - @Override - public void write(Kryo kryo, Output output) { - output.writeString(recordKey); - output.writeString(partitionPath); - } - - @Override - public void read(Kryo kryo, Input input) { - this.recordKey = input.readString(); - this.partitionPath = input.readString(); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java index a23316a2ff9fd..d78241aaeb455 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -354,25 +354,21 @@ public final void read(Kryo kryo, Input input) { public abstract HoodieRecord joinWith(HoodieRecord other, Schema targetSchema); /** - * Rewrite record into new schema(add meta columns) + * Rewrites record into new target schema containing Hudi-specific meta-fields + * + * NOTE: This operation is idempotent */ - public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException; + public abstract HoodieRecord prependMetaFields(Schema recordSchema, Schema targetSchema, MetadataValues metadataValues, Properties props); /** * Support schema evolution. */ - public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols) throws IOException; + public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map renameCols); - public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) throws IOException { + public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) { return rewriteRecordWithNewSchema(recordSchema, props, newSchema, Collections.emptyMap()); } - /** - * This method could change in the future. - * @temporary - */ - public abstract HoodieRecord updateMetadataValues(Schema recordSchema, Properties props, MetadataValues metadataValues) throws IOException; - public abstract boolean isDelete(Schema recordSchema, Properties props) throws IOException; /** @@ -391,6 +387,10 @@ public static String generateSequenceId(String instantTime, int partitionId, lon return instantTime + "_" + partitionId + "_" + recordIndex; } + protected static boolean hasMetaFields(Schema schema) { + return schema.getField(HoodieRecord.RECORD_KEY_METADATA_FIELD) != null; + } + /** * A special record returned by {@link HoodieRecordPayload}, which means we should just skip this record. * This record is only used for {@link HoodieRecordPayload} currently, so it should not diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java index 9b1c695d491ea..29e158812f485 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieTimelineTimeZone.java @@ -18,20 +18,29 @@ package org.apache.hudi.common.model; +import java.time.ZoneId; +import java.util.TimeZone; + /** * Hoodie TimelineZone. */ public enum HoodieTimelineTimeZone { - LOCAL("local"), - UTC("utc"); + LOCAL("local", ZoneId.systemDefault()), + UTC("utc", TimeZone.getTimeZone("UTC").toZoneId()); private final String timeZone; + private final ZoneId zoneId; - HoodieTimelineTimeZone(String timeZone) { + HoodieTimelineTimeZone(String timeZone, ZoneId zoneId) { this.timeZone = timeZone; + this.zoneId = zoneId; } public String getTimeZone() { return timeZone; } + + public ZoneId getZoneId() { + return zoneId; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java b/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java index baee08163380b..361da5639f48b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/MetadataValues.java @@ -18,47 +18,95 @@ package org.apache.hudi.common.model; -import java.util.HashMap; -import java.util.Map; - public class MetadataValues { - private final Map kv; - public MetadataValues() { - this.kv = new HashMap<>(); + // NOTE: These fields are laid out in the same order as they are encoded in + // each record and that should be preserved + private String commitTime; + private String commitSeqNo; + private String recordKey; + private String partitionPath; + private String fileName; + private String operation; + + private boolean set = false; + + public MetadataValues() {} + + public String getCommitTime() { + return commitTime; + } + + public String getCommitSeqNo() { + return commitSeqNo; + } + + public String getRecordKey() { + return recordKey; + } + + public String getPartitionPath() { + return partitionPath; + } + + public String getFileName() { + return fileName; + } + + public String getOperation() { + return operation; } public MetadataValues setCommitTime(String value) { - this.kv.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, value); + this.commitTime = value; + this.set = true; return this; } public MetadataValues setCommitSeqno(String value) { - this.kv.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, value); + this.commitSeqNo = value; + this.set = true; return this; } public MetadataValues setRecordKey(String value) { - this.kv.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, value); + this.recordKey = value; + this.set = true; return this; } public MetadataValues setPartitionPath(String value) { - this.kv.put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, value); + this.partitionPath = value; + this.set = true; return this; } public MetadataValues setFileName(String value) { - this.kv.put(HoodieRecord.FILENAME_METADATA_FIELD, value); + this.fileName = value; + this.set = true; return this; } public MetadataValues setOperation(String value) { - this.kv.put(HoodieRecord.OPERATION_METADATA_FIELD, value); + this.operation = value; + this.set = true; return this; } - public Map getKv() { - return kv; + public boolean isEmpty() { + return !set; + } + + public String[] getValues() { + return new String[] { + // NOTE: These fields are laid out in the same order as they are encoded in + // each record and that should be preserved + commitTime, + commitSeqNo, + recordKey, + partitionPath, + fileName, + operation + }; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java index 08c020f63cacb..aa61da9088367 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateAvroPayload.java @@ -18,17 +18,17 @@ package org.apache.hudi.common.model; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.avro.generic.IndexedRecord; + import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.avro.generic.IndexedRecord; - import java.io.IOException; import java.util.List; import java.util.Properties; @@ -85,7 +85,34 @@ * Result data after preCombine or combineAndGetUpdateValue: * id ts name price * 1 2 name_1 price_1 - * + * + * + *

Gotchas: + *

In cases where a batch of records is preCombine before combineAndGetUpdateValue with the underlying records to be updated located in parquet files, the end states of records might not be as how + * one will expect when applying a straightforward partial update. + * + *

Gotchas-Example: + *

+ *  -- Insertion order of records:
+ *  INSERT INTO t1 VALUES (1, 'a1', 10, 1000);                          -- (1)
+ *  INSERT INTO t1 VALUES (1, 'a1', 11, 999), (1, 'a1_0', null, 1001);  -- (2)
+ *
+ *  SELECT id, name, price, _ts FROM t1;
+ *  -- One would the results to return:
+ *  -- 1    a1_0    10.0    1001
+
+ *  -- However, the results returned are:
+ *  -- 1    a1_0    11.0    1001
+ *
+ *  -- This occurs as preCombine is applied on (2) first to return:
+ *  -- 1    a1_0    11.0    1001
+ *
+ *  -- And this then combineAndGetUpdateValue with the existing oldValue:
+ *  -- 1    a1_0    10.0    1000
+ *
+ *  -- To return:
+ *  -- 1    a1_0    11.0    1001
+ * 
*/ public class PartialUpdateAvroPayload extends OverwriteNonDefaultsWithLatestAvroPayload { @@ -100,14 +127,14 @@ public PartialUpdateAvroPayload(Option record) { @Override public PartialUpdateAvroPayload preCombine(OverwriteWithLatestAvroPayload oldValue, Schema schema, Properties properties) { if (oldValue.recordBytes.length == 0) { - // use natural order for delete record + // use natural order for deleted record return this; } // pick the payload with greater ordering value as insert record final boolean shouldPickOldRecord = oldValue.orderingVal.compareTo(orderingVal) > 0 ? true : false; try { GenericRecord oldRecord = HoodieAvroUtils.bytesToAvro(oldValue.recordBytes, schema); - Option mergedRecord = mergeOldRecord(oldRecord, schema, shouldPickOldRecord); + Option mergedRecord = mergeOldRecord(oldRecord, schema, shouldPickOldRecord, true); if (mergedRecord.isPresent()) { return new PartialUpdateAvroPayload((GenericRecord) mergedRecord.get(), shouldPickOldRecord ? oldValue.orderingVal : this.orderingVal); @@ -120,12 +147,12 @@ public PartialUpdateAvroPayload preCombine(OverwriteWithLatestAvroPayload oldVal @Override public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { - return this.mergeOldRecord(currentValue, schema, false); + return this.mergeOldRecord(currentValue, schema, false, false); } @Override public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties prop) throws IOException { - return mergeOldRecord(currentValue, schema, isRecordNewer(orderingVal, currentValue, prop)); + return mergeOldRecord(currentValue, schema, isRecordNewer(orderingVal, currentValue, prop), false); } /** @@ -139,18 +166,31 @@ public Boolean overwriteField(Object value, Object defaultValue) { // Utilities // ------------------------------------------------------------------------- + /** + * Merge old record with new record. + * + * @param oldRecord + * @param schema + * @param isOldRecordNewer + * @param isPreCombining flag for deleted record combine logic + * 1 preCombine: if delete record is newer, return merged record with _hoodie_is_deleted = true + * 2 combineAndGetUpdateValue: if delete record is newer, return empty since we don't need to store deleted data to storage + * @return + * @throws IOException + */ private Option mergeOldRecord(IndexedRecord oldRecord, - Schema schema, - boolean isOldRecordNewer) throws IOException { - Option recordOption = getInsertValue(schema); - if (!recordOption.isPresent()) { + Schema schema, + boolean isOldRecordNewer, boolean isPreCombining) throws IOException { + Option recordOption = getInsertValue(schema, isPreCombining); + + if (!recordOption.isPresent() && !isPreCombining) { // use natural order for delete record return Option.empty(); } if (isOldRecordNewer && schema.getField(HoodieRecord.COMMIT_TIME_METADATA_FIELD) != null) { // handling disorder, should use the metadata fields of the updating record - return mergeDisorderRecordsWithMetadata(schema, (GenericRecord) oldRecord, (GenericRecord) recordOption.get()); + return mergeDisorderRecordsWithMetadata(schema, (GenericRecord) oldRecord, (GenericRecord) recordOption.get(), isPreCombining); } else if (isOldRecordNewer) { return mergeRecords(schema, (GenericRecord) oldRecord, (GenericRecord) recordOption.get()); } else { @@ -158,20 +198,34 @@ private Option mergeOldRecord(IndexedRecord oldRecord, } } + /** + * return itself as long as it called by preCombine + * @param schema + * @param isPreCombining + * @return + * @throws IOException + */ + public Option getInsertValue(Schema schema, boolean isPreCombining) throws IOException { + if (recordBytes.length == 0 || (!isPreCombining && isDeletedRecord)) { + return Option.empty(); + } + + return Option.of((IndexedRecord) HoodieAvroUtils.bytesToAvro(recordBytes, schema)); + } + /** * Merges the given disorder records with metadata. * * @param schema The record schema * @param oldRecord The current record from file * @param updatingRecord The incoming record - * * @return the merged record option */ protected Option mergeDisorderRecordsWithMetadata( Schema schema, GenericRecord oldRecord, - GenericRecord updatingRecord) { - if (isDeleteRecord(oldRecord)) { + GenericRecord updatingRecord, boolean isPreCombining) { + if (isDeleteRecord(oldRecord) && !isPreCombining) { return Option.empty(); } else { final GenericRecordBuilder builder = new GenericRecordBuilder(schema); @@ -197,9 +251,8 @@ protected Option mergeDisorderRecordsWithMetadata( * Returns whether the given record is newer than the record of this payload. * * @param orderingVal - * @param record The record - * @param prop The payload properties - * + * @param record The record + * @param prop The payload properties * @return true if the given record is newer */ private static boolean isRecordNewer(Comparable orderingVal, IndexedRecord record, Properties prop) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index 3be3004e5364e..d4574891fda57 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -133,4 +133,19 @@ public static boolean isDataChange(WriteOperationType operation) { || operation == WriteOperationType.INSERT_OVERWRITE_TABLE || operation == WriteOperationType.BOOTSTRAP; } + + public static boolean isInsert(WriteOperationType operation) { + return operation == WriteOperationType.INSERT + || operation == WriteOperationType.INSERT_PREPPED + || operation == WriteOperationType.BULK_INSERT + || operation == WriteOperationType.BULK_INSERT_PREPPED + || operation == WriteOperationType.INSERT_OVERWRITE + || operation == WriteOperationType.INSERT_OVERWRITE_TABLE; + + } + + public static boolean isUpsert(WriteOperationType operation) { + return operation == WriteOperationType.UPSERT + || operation == WriteOperationType.UPSERT_PREPPED; + } } 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..31fc8ad79c17b 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 @@ -38,7 +38,7 @@ * - For inserts, op=i * - For deletes, op=d * - For updates, op=u - * - For snapshort inserts, op=r + * - For snapshot inserts, op=r *

* This payload implementation will issue matching insert, delete, updates against the hudi table */ @@ -56,15 +56,20 @@ 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 = (recordBytes.length == 0) + ? Option.empty() : Option.of((IndexedRecord) HoodieAvroUtils.bytesToAvro(recordBytes, 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) @@ -77,14 +82,23 @@ private Option handleDeleteOperation(IndexedRecord insertRecord) boolean delete = false; if (insertRecord instanceof GenericRecord) { GenericRecord record = (GenericRecord) insertRecord; - Object value = HoodieAvroUtils.getFieldVal(record, DebeziumConstants.FLATTENED_OP_COL_NAME); - delete = value != null && value.toString().equalsIgnoreCase(DebeziumConstants.DELETE_OP); + delete = isDebeziumDeleteRecord(record); } 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); + } + + @Override + protected boolean isDeleteRecord(GenericRecord record) { + return isDebeziumDeleteRecord(record) || super.isDeleteRecord(record); + } + + private static boolean isDebeziumDeleteRecord(GenericRecord record) { + Object value = HoodieAvroUtils.getFieldVal(record, DebeziumConstants.FLATTENED_OP_COL_NAME); + return value != null && value.toString().equalsIgnoreCase(DebeziumConstants.DELETE_OP); } -} \ No newline at end of file +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index e450614e2b61b..1aff3d6e2e96d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -20,8 +20,6 @@ import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex; -import org.apache.hudi.common.config.ConfigClassProperty; -import org.apache.hudi.common.config.ConfigGroups; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.OrderedProperties; @@ -64,6 +62,9 @@ import java.util.stream.Collectors; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before_after; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.op_key_only; /** * Configurations on the Hoodie Table like type of ingestion, storage formats, hive table name etc Configurations are loaded from hoodie.properties, these properties are usually set during @@ -72,14 +73,6 @@ * @see HoodieTableMetaClient * @since 0.3.0 */ -@ConfigClassProperty(name = "Table Configurations", - groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "Configurations that persist across writes and read on a Hudi table " - + " like base, log file formats, table name, creation schema, table version layouts. " - + " Configurations are loaded from hoodie.properties, these properties are usually set during " - + "initializing a path as hoodie base path and rarely changes during " - + "the lifetime of the table. Writers/Queries' configurations are validated against these " - + " each time for compatibility.") public class HoodieTableConfig extends HoodieConfig { private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class); @@ -137,15 +130,15 @@ public class HoodieTableConfig extends HoodieConfig { public static final ConfigProperty CDC_SUPPLEMENTAL_LOGGING_MODE = ConfigProperty .key("hoodie.table.cdc.supplemental.logging.mode") - .defaultValue(HoodieCDCSupplementalLoggingMode.OP_KEY.getValue()) + .defaultValue(data_before_after.name()) .withValidValues( - HoodieCDCSupplementalLoggingMode.OP_KEY.getValue(), - HoodieCDCSupplementalLoggingMode.WITH_BEFORE.getValue(), - HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER.getValue()) + op_key_only.name(), + data_before.name(), + data_before_after.name()) .sinceVersion("0.13.0") - .withDocumentation("When 'cdc_op_key' persist the 'op' and the record key only," - + " when 'cdc_data_before' persist the additional 'before' image ," - + " and when 'cdc_data_before_after', persist the 'before' and 'after' at the same time."); + .withDocumentation("Setting 'op_key_only' persists the 'op' and the record key only, " + + "setting 'data_before' persists the additional 'before' image, " + + "and setting 'data_before_after' persists the additional 'before' and 'after' images."); public static final ConfigProperty CREATE_SCHEMA = ConfigProperty .key("hoodie.table.create.schema") @@ -659,13 +652,17 @@ public boolean isCDCEnabled() { } public HoodieCDCSupplementalLoggingMode cdcSupplementalLoggingMode() { - return HoodieCDCSupplementalLoggingMode.parse(getStringOrDefault(CDC_SUPPLEMENTAL_LOGGING_MODE)); + return HoodieCDCSupplementalLoggingMode.valueOf(getStringOrDefault(CDC_SUPPLEMENTAL_LOGGING_MODE)); } public String getKeyGeneratorClassName() { return getString(KEY_GENERATOR_CLASS_NAME); } + public HoodieTimelineTimeZone getTimelineTimezone() { + return HoodieTimelineTimeZone.valueOf(getStringOrDefault(TIMELINE_TIMEZONE)); + } + public String getHiveStylePartitioningEnable() { return getStringOrDefault(HIVE_STYLE_PARTITIONING_ENABLE); } 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 e7c749a2911b2..00fa1b97db06d 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 @@ -338,6 +338,10 @@ public Configuration getHadoopConf() { return hadoopConf.get(); } + public SerializableConfiguration getSerializableHadoopConf() { + return hadoopConf; + } + /** * Get the active instants as a timeline. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 4eddd6df0312b..03ed542bd609e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.table; -import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; @@ -31,7 +30,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.Functions.Function1; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; @@ -315,40 +313,6 @@ public Option getTableAvroSchemaFromLatestCommit(boolean includeMetadata return Option.empty(); } - /** - * Get latest schema either from incoming schema or table schema. - * @param writeSchema incoming batch's write schema. - * @param convertTableSchemaToAddNamespace {@code true} if table schema needs to be converted. {@code false} otherwise. - * @param converterFn converter function to be called over table schema (to add namespace may be). Each caller can decide if any conversion is required. - * @return the latest schema. - * - * @deprecated will be removed (HUDI-4472) - */ - @Deprecated - public Schema getLatestSchema(Schema writeSchema, boolean convertTableSchemaToAddNamespace, - Function1 converterFn) { - Schema latestSchema = writeSchema; - try { - if (metaClient.isTimelineNonEmpty()) { - Schema tableSchema = getTableAvroSchemaWithoutMetadataFields(); - if (convertTableSchemaToAddNamespace && converterFn != null) { - tableSchema = converterFn.apply(tableSchema); - } - if (writeSchema.getFields().size() < tableSchema.getFields().size() && AvroSchemaUtils.isSchemaCompatible(writeSchema, tableSchema)) { - // if incoming schema is a subset (old schema) compared to table schema. For eg, one of the - // ingestion pipeline is still producing events in old schema - latestSchema = tableSchema; - LOG.debug("Using latest table schema to rewrite incoming records " + tableSchema.toString()); - } - } - } catch (IllegalArgumentException | InvalidTableException e) { - LOG.warn("Could not find any commits, falling back to using incoming batch's write schema"); - } catch (Exception e) { - LOG.warn("Unknown exception thrown " + e.getMessage() + ", Falling back to using incoming batch's write schema"); - } - return latestSchema; - } - private MessageType readSchemaFromParquetBaseFile(Path parquetFilePath) throws IOException { LOG.info("Reading schema from " + parquetFilePath); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java index 994b7ea477e0e..6ca116015b030 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCExtractor.java @@ -56,11 +56,13 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.AS_IS; -import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_DELETE; -import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.BASE_FILE_INSERT; -import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.LOG_FILE; -import static org.apache.hudi.common.table.cdc.HoodieCDCInferCase.REPLACE_COMMIT; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.AS_IS; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.BASE_FILE_DELETE; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.BASE_FILE_INSERT; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.LOG_FILE; +import static org.apache.hudi.common.table.cdc.HoodieCDCInferenceCase.REPLACE_COMMIT; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before_after; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.op_key_only; 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; @@ -280,7 +282,7 @@ private HoodieCDCFileSplit parseWriteStat( } } else { // this is a cdc log - if (supplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER)) { + if (supplementalLoggingMode == data_before_after) { cdcFileSplit = new HoodieCDCFileSplit(instantTs, AS_IS, writeStat.getCdcStats().keySet()); } else { try { @@ -292,7 +294,7 @@ private HoodieCDCFileSplit parseWriteStat( FileSlice beforeFileSlice = null; FileSlice currentFileSlice = new FileSlice(fileGroupId, instant.getTimestamp(), new HoodieBaseFile(fs.getFileStatus(new Path(basePath, writeStat.getPath()))), new ArrayList<>()); - if (supplementalLoggingMode.equals(HoodieCDCSupplementalLoggingMode.OP_KEY)) { + if (supplementalLoggingMode == op_key_only) { beforeFileSlice = new FileSlice(fileGroupId, writeStat.getPrevCommit(), beforeBaseFile, new ArrayList<>()); } cdcFileSplit = new HoodieCDCFileSplit(instantTs, AS_IS, writeStat.getCdcStats().keySet(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java index d508f7ac4eab3..f992a9b228c60 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCFileSplit.java @@ -33,16 +33,16 @@ * This contains all the information that retrieve the change data at a single file group and * at a single commit. *

- * For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_INSERT}, `cdcFile` is a current version of + * For `cdcInferCase` = {@link HoodieCDCInferenceCase#BASE_FILE_INSERT}, `cdcFile` is a current version of * the base file in the group, and `beforeFileSlice` is None. - * For `cdcInferCase` = {@link HoodieCDCInferCase#BASE_FILE_DELETE}, `cdcFile` is null, + * For `cdcInferCase` = {@link HoodieCDCInferenceCase#BASE_FILE_DELETE}, `cdcFile` is null, * `beforeFileSlice` is the previous version of the base file in the group. - * For `cdcInferCase` = {@link HoodieCDCInferCase#AS_IS}, `cdcFile` is a log file with cdc blocks. + * For `cdcInferCase` = {@link HoodieCDCInferenceCase#AS_IS}, `cdcFile` is a log file with cdc blocks. * when enable the supplemental logging, both `beforeFileSlice` and `afterFileSlice` are None, * otherwise these two are the previous and current version of the base file. - * For `cdcInferCase` = {@link HoodieCDCInferCase#LOG_FILE}, `cdcFile` is a normal log file and + * For `cdcInferCase` = {@link HoodieCDCInferenceCase#LOG_FILE}, `cdcFile` is a normal log file and * `beforeFileSlice` is the previous version of the file slice. - * For `cdcInferCase` = {@link HoodieCDCInferCase#REPLACE_COMMIT}, `cdcFile` is null, + * For `cdcInferCase` = {@link HoodieCDCInferenceCase#REPLACE_COMMIT}, `cdcFile` is null, * `beforeFileSlice` is the current version of the file slice. */ public class HoodieCDCFileSplit implements Serializable, Comparable { @@ -54,7 +54,7 @@ public class HoodieCDCFileSplit implements Serializable, Comparable afterFileSlice; - public HoodieCDCFileSplit(String instant, HoodieCDCInferCase cdcInferCase, String cdcFile) { + public HoodieCDCFileSplit(String instant, HoodieCDCInferenceCase cdcInferCase, String cdcFile) { this(instant, cdcInferCase, cdcFile, Option.empty(), Option.empty()); } - public HoodieCDCFileSplit(String instant, HoodieCDCInferCase cdcInferCase, Collection cdcFiles) { + public HoodieCDCFileSplit(String instant, HoodieCDCInferenceCase cdcInferCase, Collection cdcFiles) { this(instant, cdcInferCase, cdcFiles, Option.empty(), Option.empty()); } public HoodieCDCFileSplit( String instant, - HoodieCDCInferCase cdcInferCase, + HoodieCDCInferenceCase cdcInferCase, String cdcFile, Option beforeFileSlice, Option afterFileSlice) { @@ -90,7 +90,7 @@ public HoodieCDCFileSplit( public HoodieCDCFileSplit( String instant, - HoodieCDCInferCase cdcInferCase, + HoodieCDCInferenceCase cdcInferCase, Collection cdcFiles, Option beforeFileSlice, Option afterFileSlice) { @@ -106,7 +106,7 @@ public String getInstant() { return this.instant; } - public HoodieCDCInferCase getCdcInferCase() { + public HoodieCDCInferenceCase getCdcInferCase() { return this.cdcInferCase; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferenceCase.java similarity index 91% rename from hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java rename to hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferenceCase.java index c6005c601012a..9f6d85108c1b5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferCase.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCInferenceCase.java @@ -24,10 +24,10 @@ * * AS_IS: * For this type, there must be a real cdc log file from which we get the whole/part change data. - * when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before_after', it keeps all the fields about the + * when `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#data_before_after}, it keeps all the fields about the * change data, including `op`, `ts_ms`, `before` and `after`. So read it and return directly, * no more other files need to be loaded. - * when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before', it keeps the `op`, the key and the + * when `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#data_before}, it keeps the `op`, the key and the * `before` of the changing record. When `op` is equal to 'i' or 'u', need to get the current record from the * current base/log file as `after`. * when `hoodie.table.cdc.supplemental.logging.mode` is 'op_key', it just keeps the `op` and the key of @@ -42,7 +42,7 @@ * file is new-coming, so we can load this, mark all the records with `i`, and treat them as * the value of `after`. The value of `before` for each record is null. * - * BASE_FILE_INSERT: + * BASE_FILE_DELETE: * For this type, there must be an empty file at the current instant, but a non-empty base file * at the previous instant. First we find this base file that has the same file group and belongs * to the previous instant. Then load this, mark all the records with `d`, and treat them as @@ -67,7 +67,7 @@ * a whole file group. First we find this file group. Then load this, mark all the records with * `d`, and treat them as the value of `before`. The value of `after` for each record is null. */ -public enum HoodieCDCInferCase { +public enum HoodieCDCInferenceCase { AS_IS, BASE_FILE_INSERT, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCSupplementalLoggingMode.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCSupplementalLoggingMode.java index 13a51a4f07295..b52d1432fc11a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCSupplementalLoggingMode.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCSupplementalLoggingMode.java @@ -18,8 +18,6 @@ package org.apache.hudi.common.table.cdc; -import org.apache.hudi.exception.HoodieNotSupportedException; - /** * Change log capture supplemental logging mode. The supplemental log is used for * accelerating the generation of change log details. @@ -27,36 +25,13 @@ *

Three modes are supported:

* *
    - *
  • OP_KEY: record keys, the reader needs to figure out the update before image and after image;
  • - *
  • WITH_BEFORE: before images, the reader needs to figure out the update after images;
  • - *
  • WITH_BEFORE_AFTER: before and after images, the reader can generate the details directly from the log.
  • + *
  • op_key_only: record keys, the reader needs to figure out the update before image and after image;
  • + *
  • data_before: before images, the reader needs to figure out the update after images;
  • + *
  • data_before_after: before and after images, the reader can generate the details directly from the log.
  • *
*/ public enum HoodieCDCSupplementalLoggingMode { - OP_KEY("cdc_op_key"), - WITH_BEFORE("cdc_data_before"), - WITH_BEFORE_AFTER("cdc_data_before_after"); - - private final String value; - - HoodieCDCSupplementalLoggingMode(String value) { - this.value = value; - } - - public String getValue() { - return this.value; - } - - public static HoodieCDCSupplementalLoggingMode parse(String value) { - switch (value) { - case "cdc_op_key": - return OP_KEY; - case "cdc_data_before": - return WITH_BEFORE; - case "cdc_data_before_after": - return WITH_BEFORE_AFTER; - default: - throw new HoodieNotSupportedException("Unsupported value: " + value); - } - } + op_key_only, + data_before, + data_before_after } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java index 6ca5869fdfd49..069567208b024 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/cdc/HoodieCDCUtils.java @@ -59,7 +59,7 @@ public class HoodieCDCUtils { }; /** - * The schema of cdc log file in the case `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_op_key'. + * The schema of cdc log file in the case `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#op_key_only}. */ public static final String CDC_SCHEMA_OP_AND_RECORDKEY_STRING = "{\"type\":\"record\",\"name\":\"Record\"," + "\"fields\":[" @@ -73,11 +73,11 @@ public class HoodieCDCUtils { public static Schema schemaBySupplementalLoggingMode( HoodieCDCSupplementalLoggingMode supplementalLoggingMode, Schema tableSchema) { - if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.OP_KEY) { + if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.op_key_only) { return CDC_SCHEMA_OP_AND_RECORDKEY; - } else if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.WITH_BEFORE) { + } else if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.data_before) { return createCDCSchema(tableSchema, false); - } else if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER) { + } else if (supplementalLoggingMode == HoodieCDCSupplementalLoggingMode.data_before_after) { return createCDCSchema(tableSchema, true); } else { throw new HoodieException("not support this supplemental logging mode: " + supplementalLoggingMode); @@ -109,7 +109,7 @@ private static Schema createCDCSchema(Schema tableSchema, boolean withAfterImage } /** - * Build the cdc record which has all the cdc fields when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before_after'. + * Build the cdc record which has all the cdc fields when `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#data_before_after}. */ public static GenericData.Record cdcRecord(Schema cdcSchema, String op, String commitTime, GenericRecord before, GenericRecord after) { @@ -122,7 +122,7 @@ public static GenericData.Record cdcRecord(Schema cdcSchema, String op, String c } /** - * Build the cdc record when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_data_before'. + * Build the cdc record when `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#data_before}. */ public static GenericData.Record cdcRecord(Schema cdcSchema, String op, String recordKey, GenericRecord before) { @@ -134,7 +134,7 @@ public static GenericData.Record cdcRecord(Schema cdcSchema, String op, } /** - * Build the cdc record when `hoodie.table.cdc.supplemental.logging.mode` is 'cdc_op_key'. + * Build the cdc record when `hoodie.table.cdc.supplemental.logging.mode` is {@link HoodieCDCSupplementalLoggingMode#op_key_only}. */ public static GenericData.Record cdcRecord(Schema cdcSchema, String op, String recordKey) { GenericData.Record record = new GenericData.Record(cdcSchema); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 83172ecb7aecb..fd8b4fb4377d6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -149,7 +149,7 @@ public abstract class AbstractHoodieLogRecordReader { // Collect all the block instants after scanning all the log files. private final List validBlockInstants = new ArrayList<>(); // Use scanV2 method. - private final boolean useScanV2; + private final boolean enableOptimizedLogBlocksScan; protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, @@ -158,7 +158,7 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List partitionNameOverride, InternalSchema internalSchema, Option keyFieldOverride, - boolean useScanV2, + boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger) { this.readerSchema = readerSchema; this.latestInstantTime = latestInstantTime; @@ -184,7 +184,7 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List keySpecOpt, boolean skipProcessingBlocks) { synchronized (this) { - if (useScanV2) { + if (enableOptimizedLogBlocksScan) { scanInternalV2(keySpecOpt, skipProcessingBlocks); } else { scanInternalV1(keySpecOpt); @@ -307,45 +307,43 @@ private void scanInternalV1(Option keySpecOpt) { // written per ingestion batch for a file but in reality we need to rollback (B1 & B2) // The following code ensures the same rollback block (R1) is used to rollback // both B1 & B2 - LOG.info("Reading a command block from file " + logFile.getPath()); // This is a command block - take appropriate action based on the command HoodieCommandBlock commandBlock = (HoodieCommandBlock) logBlock; String targetInstantForCommandBlock = logBlock.getLogBlockHeader().get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME); + LOG.info(String.format("Reading a command block %s with targetInstantTime %s from file %s", commandBlock.getType(), targetInstantForCommandBlock, + logFile.getPath())); switch (commandBlock.getType()) { // there can be different types of command blocks case ROLLBACK_BLOCK: - // Rollback the last read log block - // Get commit time from last record block, compare with targetCommitTime, + // Rollback older read log block(s) + // Get commit time from older record blocks, compare with targetCommitTime, // rollback only if equal, this is required in scenarios of invalid/extra // rollback blocks written due to failures during the rollback operation itself // and ensures the same rollback block (R1) is used to rollback both B1 & B2 with - // same instant_time - int numBlocksRolledBack = 0; - totalRollbacks.incrementAndGet(); - while (!currentInstantLogBlocks.isEmpty()) { - HoodieLogBlock lastBlock = currentInstantLogBlocks.peek(); + // same instant_time. + final int instantLogBlockSizeBeforeRollback = currentInstantLogBlocks.size(); + currentInstantLogBlocks.removeIf(block -> { // handle corrupt blocks separately since they may not have metadata - if (lastBlock.getBlockType() == CORRUPT_BLOCK) { + if (block.getBlockType() == CORRUPT_BLOCK) { LOG.info("Rolling back the last corrupted log block read in " + logFile.getPath()); - currentInstantLogBlocks.pop(); - numBlocksRolledBack++; - } else if (targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) { - // rollback last data block or delete block - LOG.info("Rolling back the last log block read in " + logFile.getPath()); - currentInstantLogBlocks.pop(); - numBlocksRolledBack++; - } else if (!targetInstantForCommandBlock - .contentEquals(currentInstantLogBlocks.peek().getLogBlockHeader().get(INSTANT_TIME))) { - // invalid or extra rollback block - LOG.warn("TargetInstantTime " + targetInstantForCommandBlock - + " invalid or extra rollback command block in " + logFile.getPath()); - break; - } else { - // this should not happen ideally - LOG.warn("Unable to apply rollback command block in " + logFile.getPath()); + return true; } - } + if (targetInstantForCommandBlock.contentEquals(block.getLogBlockHeader().get(INSTANT_TIME))) { + // rollback older data block or delete block + LOG.info(String.format("Rolling back an older log block read from %s with instantTime %s", + logFile.getPath(), targetInstantForCommandBlock)); + return true; + } + return false; + }); + + final int numBlocksRolledBack = instantLogBlockSizeBeforeRollback - currentInstantLogBlocks.size(); + totalRollbacks.addAndGet(numBlocksRolledBack); LOG.info("Number of applied rollback blocks " + numBlocksRolledBack); + if (numBlocksRolledBack == 0) { + LOG.warn(String.format("TargetInstantTime %s invalid or extra rollback command block in %s", + targetInstantForCommandBlock, logFile.getPath())); + } break; default: throw new UnsupportedOperationException("Command type not yet supported."); @@ -842,16 +840,11 @@ private Option, Schema>> composeEvolve Schema mergedAvroSchema = AvroInternalSchemaConverter.convert(mergedInternalSchema, readerSchema.getFullName()); return Option.of(Pair.of((record) -> { - try { - return record.rewriteRecordWithNewSchema( - dataBlock.getSchema(), - this.hoodieTableMetaClient.getTableConfig().getProps(), - mergedAvroSchema, - Collections.emptyMap()); - } catch (IOException e) { - LOG.error("Error rewrite record with new schema", e); - throw new HoodieException(e); - } + return record.rewriteRecordWithNewSchema( + dataBlock.getSchema(), + this.hoodieTableMetaClient.getTableConfig().getProps(), + mergedAvroSchema, + Collections.emptyMap()); }, mergedAvroSchema)); } @@ -894,7 +887,7 @@ public Builder withRecordMerger(HoodieRecordMerger recordMerger) { throw new UnsupportedOperationException(); } - public Builder withUseScanV2(boolean useScanV2) { + public Builder withOptimizedLogBlocksScan(boolean enableOptimizedLogBlocksScan) { throw new UnsupportedOperationException(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java index c41d78b280842..1a256956bfc98 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java @@ -47,7 +47,6 @@ import javax.annotation.concurrent.NotThreadSafe; import java.io.Closeable; import java.io.IOException; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -98,9 +97,9 @@ private HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List partitionName, InternalSchema internalSchema, Option keyFieldOverride, - boolean useScanV2, HoodieRecordMerger recordMerger) { + boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, - instantRange, withOperationField, forceFullScan, partitionName, internalSchema, keyFieldOverride, useScanV2, recordMerger); + instantRange, withOperationField, forceFullScan, partitionName, internalSchema, keyFieldOverride, enableOptimizedLogBlocksScan, recordMerger); try { this.maxMemorySizeInBytes = maxMemorySizeInBytes; // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize @@ -215,7 +214,7 @@ public Iterator iterator() { } public Map getRecords() { - return Collections.unmodifiableMap(records); + return records; } public HoodieRecordType getRecordType() { @@ -333,7 +332,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { // By default, we're doing a full-scan private boolean forceFullScan = true; // Use scanV2 method. - private boolean useScanV2 = false; + private boolean enableOptimizedLogBlocksScan = false; private HoodieRecordMerger recordMerger; @Override @@ -430,8 +429,8 @@ public Builder withPartition(String partitionName) { } @Override - public Builder withUseScanV2(boolean useScanV2) { - this.useScanV2 = useScanV2; + public Builder withOptimizedLogBlocksScan(boolean enableOptimizedLogBlocksScan) { + this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan; return this; } @@ -462,7 +461,7 @@ public HoodieMergedLogRecordScanner build() { latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader, bufferSize, spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, forceFullScan, - Option.ofNullable(partitionName), internalSchema, Option.ofNullable(keyFieldOverride), useScanV2, recordMerger); + Option.ofNullable(partitionName), internalSchema, Option.ofNullable(keyFieldOverride), enableOptimizedLogBlocksScan, recordMerger); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java index 2a7c91641e11b..726172e5ee0ee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java @@ -42,9 +42,9 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize, LogRecordScannerCallback callback, Option instantRange, InternalSchema internalSchema, - boolean useScanV2, HoodieRecordMerger recordMerger) { + boolean enableOptimizedLogBlocksScan, HoodieRecordMerger recordMerger) { super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, - false, true, Option.empty(), internalSchema, Option.empty(), useScanV2, recordMerger); + false, true, Option.empty(), internalSchema, Option.empty(), enableOptimizedLogBlocksScan, recordMerger); this.callback = callback; } @@ -105,7 +105,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { private Option instantRange = Option.empty(); // specific configurations private LogRecordScannerCallback callback; - private boolean useScanV2; + private boolean enableOptimizedLogBlocksScan; private HoodieRecordMerger recordMerger; public Builder withFileSystem(FileSystem fs) { @@ -167,8 +167,8 @@ public Builder withLogRecordScannerCallback(LogRecordScannerCallback callback) { } @Override - public Builder withUseScanV2(boolean useScanV2) { - this.useScanV2 = useScanV2; + public Builder withOptimizedLogBlocksScan(boolean enableOptimizedLogBlocksScan) { + this.enableOptimizedLogBlocksScan = enableOptimizedLogBlocksScan; return this; } @@ -184,7 +184,7 @@ public HoodieUnMergedLogRecordScanner build() { return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange, - internalSchema, useScanV2, recordMerger); + internalSchema, enableOptimizedLogBlocksScan, recordMerger); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java index afc7937f2171a..259ba284a8973 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java @@ -170,9 +170,7 @@ private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) } public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException { - // Get schema from the header - Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - return new RecordIterator(dataBlock.readerSchema, writerSchema, content); + return new RecordIterator(dataBlock.readerSchema, dataBlock.getSchemaFromHeader(), content); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java index 17e4d7a08e274..4fabe6afaf979 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; @@ -62,6 +63,9 @@ public abstract class HoodieDataBlock extends HoodieLogBlock { protected Schema readerSchema; + // Map of string schema to parsed schema. + private static ConcurrentHashMap schemaMap = new ConcurrentHashMap<>(); + /** * NOTE: This ctor is used on the write-path (ie when records ought to be written into the log) */ @@ -194,6 +198,12 @@ protected Option getRecordKey(HoodieRecord record) { return Option.ofNullable(record.getRecordKey(readerSchema, keyFieldName)); } + protected Schema getSchemaFromHeader() { + String schemaStr = getLogBlockHeader().get(HeaderMetadataType.SCHEMA); + schemaMap.computeIfAbsent(schemaStr, (schemaString) -> new Schema.Parser().parse(schemaString)); + return schemaMap.get(schemaStr); + } + /** * Converts the given list to closable iterator. */ diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java index a5168072d014d..d920495f97173 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java @@ -68,6 +68,7 @@ public byte[] getContentBytes() throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream output = new DataOutputStream(baos); + // TODO(HUDI-5760) avoid using Kryo for serialization here byte[] bytesToWrite = SerializationUtils.serialize(getRecordsToDelete()); output.writeInt(version); output.writeInt(bytesToWrite.length); @@ -97,6 +98,7 @@ public DeleteRecord[] getRecordsToDelete() { } } + // TODO(HUDI-5760) avoid using Kryo for serialization here private static DeleteRecord[] deserialize(int version, byte[] data) { if (version == 1) { // legacy version diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java index 5de47ab70638c..ca3f8152cdb7e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java @@ -165,11 +165,8 @@ protected byte[] serializeRecords(List records) throws IOException protected ClosableIterator> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException { checkState(readerSchema != null, "Reader's schema has to be non-null"); - // Get schema from the header - Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA)); - // Read the content - HoodieAvroHFileReader reader = new HoodieAvroHFileReader(null, pathForReader, content, Option.of(writerSchema)); + HoodieAvroHFileReader reader = new HoodieAvroHFileReader(null, pathForReader, content, Option.of(getSchemaFromHeader())); return unsafeCast(reader.getRecordIterator(readerSchema)); } @@ -196,7 +193,8 @@ protected ClosableIterator> lookupRecords(List keys, Collections.sort(sortedKeys); final HoodieAvroHFileReader reader = - new HoodieAvroHFileReader(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf)); + new HoodieAvroHFileReader(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf), + Option.of(getSchemaFromHeader())); // Get writer's schema from the header final ClosableIterator> recordIterator = diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 0999778fb5392..9c9f40786c02d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -142,7 +142,11 @@ public void loadInstantDetailsInMemory(String startTs, String endTs) { public void loadCompletedInstantDetailsInMemory() { loadInstants(null, true, - record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString())); + record -> { + // Very old archived instants don't have action state set. + Object action = record.get(ACTION_STATE); + return action == null || HoodieInstant.State.COMPLETED.toString().equals(action.toString()); + }); } public void loadCompactionDetailsInMemory(String compactionInstantTime) { @@ -151,9 +155,13 @@ public void loadCompactionDetailsInMemory(String compactionInstantTime) { public void loadCompactionDetailsInMemory(String startTs, String endTs) { // load compactionPlan - loadInstants(new TimeRangeFilter(startTs, endTs), true, record -> - record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION) - && HoodieInstant.State.INFLIGHT.toString().equals(record.get(ACTION_STATE).toString()) + loadInstants(new TimeRangeFilter(startTs, endTs), true, + record -> { + // Older files don't have action state set. + Object action = record.get(ACTION_STATE); + return record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION) + && (action == null || HoodieInstant.State.INFLIGHT.toString().equals(action.toString())); + } ); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java new file mode 100644 index 0000000000000..ef5a886948765 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DTOUtils.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.dto; + +import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.util.ValidationUtils.checkState; + +/** + * DTO utils to hold batch apis. + */ +public class DTOUtils { + + public static List fileGroupDTOsfromFileGroups(List fileGroups) { + if (fileGroups.isEmpty()) { + return Collections.emptyList(); + } else if (fileGroups.size() == 1) { + return Collections.singletonList(FileGroupDTO.fromFileGroup(fileGroups.get(0), true)); + } else { + List fileGroupDTOS = new ArrayList<>(); + fileGroupDTOS.add(FileGroupDTO.fromFileGroup(fileGroups.get(0), true)); + fileGroupDTOS.addAll(fileGroups.subList(1, fileGroups.size()).stream() + .map(fg -> FileGroupDTO.fromFileGroup(fg, false)).collect(Collectors.toList())); + return fileGroupDTOS; + } + } + + public static Stream fileGroupDTOsToFileGroups(List dtos, HoodieTableMetaClient metaClient) { + if (dtos.isEmpty()) { + return Stream.empty(); + } + + // Timeline exists only in the first file group DTO. Optimisation to reduce payload size. + checkState(dtos.get(0).timeline != null, "Timeline is expected to be set for the first FileGroupDTO"); + HoodieTimeline timeline = TimelineDTO.toTimeline(dtos.get(0).timeline, metaClient); + return dtos.stream().map(dto -> FileGroupDTO.toFileGroup(dto, timeline)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java index dfbd40126c07a..bc5cbdb8022a0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/FileGroupDTO.java @@ -19,7 +19,7 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.model.HoodieFileGroup; -import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; @@ -45,19 +45,20 @@ public class FileGroupDTO { @JsonProperty("timeline") TimelineDTO timeline; - public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) { + public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup, boolean includeTimeline) { FileGroupDTO dto = new FileGroupDTO(); dto.partition = fileGroup.getPartitionPath(); dto.id = fileGroup.getFileGroupId().getFileId(); dto.slices = fileGroup.getAllRawFileSlices().map(FileSliceDTO::fromFileSlice).collect(Collectors.toList()); - dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline()); + if (includeTimeline) { + dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline()); + } return dto; } - public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTableMetaClient metaClient) { - HoodieFileGroup fileGroup = - new HoodieFileGroup(dto.partition, dto.id, TimelineDTO.toTimeline(dto.timeline, metaClient)); - dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileSlice -> fileGroup.addFileSlice(fileSlice)); + public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTimeline fgTimeline) { + HoodieFileGroup fileGroup = new HoodieFileGroup(dto.partition, dto.id, fgTimeline); + dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileGroup::addFileSlice); return fileGroup; } } 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 8ea017977e985..37c43e56a6d85 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 @@ -73,11 +73,15 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; /** - * Common thread-safe implementation for multiple TableFileSystemView Implementations. Provides uniform handling of (a) - * Loading file-system views from underlying file-system (b) Pending compaction operations and changing file-system - * views based on that (c) Thread-safety in loading and managing file system views for this table. (d) resetting - * file-system views The actual mechanism of fetching file slices from different view storages is delegated to - * sub-classes. + * Common thread-safe implementation for multiple TableFileSystemView Implementations. + * Provides uniform handling of: + *
    + *
  • Loading file-system views from underlying file-system;
  • + *
  • Pending compaction operations and changing file-system views based on that;
  • + *
  • Thread-safety in loading and managing file system views for this table;
  • + *
  • resetting file-system views.
  • + *
+ * The actual mechanism of fetching file slices from different view storages is delegated to sub-classes. */ public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable { @@ -95,8 +99,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; @@ -247,7 +251,9 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { } }); - Map replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + // Duplicate key error when insert_overwrite same partition in multi writer, keep the instant with greater timestamp when the file group id conflicts + Map replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, + (instance1, instance2) -> HoodieTimeline.compareTimestamps(instance1.getTimestamp(), HoodieTimeline.LESSER_THAN, instance2.getTimestamp()) ? instance2 : instance1)); resetReplacedFileGroups(replacedFileGroups); LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + replacedTimeline.countInstants() + " instants, " + replacedFileGroups.size() + " replaced file groups"); @@ -257,6 +263,8 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { public void close() { try { writeLock.lock(); + this.metaClient = null; + this.visibleCommitsAndCompactionTimeline = null; clear(); } finally { writeLock.unlock(); @@ -265,6 +273,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() { @@ -281,7 +291,7 @@ public void reset() { /** * Clear the resource. */ - private void clear() { + protected void clear() { addedPartitions.clear(); resetViewState(); bootstrapIndex = null; @@ -1384,32 +1394,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().filterCompletedOrMajorOrMinorCompactionInstants(); try { writeLock.lock(); - runSync(oldTimeline, newTimeline); + HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedOrMajorOrMinorCompactionInstants(); + 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/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index ea72f305b0738..e6b00ca0d8e7f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -430,11 +430,12 @@ public Stream fetchLatestFileSlicesIncludingInflight(String partition public void close() { super.close(); this.fgIdToPendingCompaction = null; + this.fgIdToPendingLogCompaction = null; this.partitionToFileGroupsMap = null; this.fgIdToBootstrapBaseFile = null; this.fgIdToReplaceInstants = null; this.fgIdToPendingClustering = null; - closed = true; + this.closed = true; } @Override 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 ea49cfb54a82b..33f8a359a0f8a 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 @@ -78,7 +78,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().filterCompletedOrMajorOrMinorCompactionInstants(); try { if (incrementalTimelineSyncEnabled) { TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline); @@ -94,14 +105,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/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index bffd6353aa719..adb33d08ac851 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 @@ -29,6 +29,7 @@ import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; +import org.apache.hudi.common.table.timeline.dto.DTOUtils; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.timeline.dto.InstantDTO; @@ -398,7 +399,7 @@ public Stream getAllFileGroups(String partitionPath) { try { List fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap, new TypeReference>() {}, RequestMethod.GET); - return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -410,7 +411,7 @@ public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitT try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON, paramsMap, new TypeReference>() {}, RequestMethod.GET); - return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -422,7 +423,7 @@ public Stream getReplacedFileGroupsBefore(String maxCommitTime, try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE, paramsMap, new TypeReference>() {}, RequestMethod.GET); - return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -434,7 +435,7 @@ public Stream getAllReplacedFileGroups(String partitionPath) { try { List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_PARTITION, paramsMap, new TypeReference>() {}, RequestMethod.GET); - return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + return DTOUtils.fileGroupDTOsToFileGroups(fileGroups, metaClient); } catch (IOException e) { throw new HoodieRemoteException(e); } 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 c669de76b2138..b8ce42dc5a53f 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 @@ -18,6 +18,7 @@ package org.apache.hudi.common.util; +import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieClusteringStrategy; @@ -227,8 +228,8 @@ public static boolean isPendingClusteringInstant(HoodieTableMetaClient metaClien } /** - * Checks whether the latest clustering instant has a subsequent cleaning action. Returns - * the clustering instant if there is such cleaning action or empty. + * Returns the oldest instant to retain. + * Make sure the clustering instant won't be archived before cleaned, and the oldest inflight clustering instant has a previous commit. * * @param activeTimeline The active timeline * @param metaClient The meta client @@ -236,29 +237,51 @@ public static boolean isPendingClusteringInstant(HoodieTableMetaClient metaClien */ public static Option getOldestInstantToRetainForClustering( HoodieActiveTimeline activeTimeline, HoodieTableMetaClient metaClient) throws IOException { - HoodieTimeline replaceTimeline = activeTimeline.getCompletedReplaceTimeline(); + 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(); + activeTimeline.getCleanerTimeline().filterCompletedInstants().lastInstant(); 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(); - return StringUtils.isNullOrEmpty(earliestCommitToRetain) - ? Option.empty() - : replaceTimeline.filter(instant -> - HoodieTimeline.compareTimestamps(instant.getTimestamp(), + HoodieActionInstant earliestInstantToRetain = CleanerUtils.getCleanerPlan(metaClient, cleanInstant.isRequested() + ? cleanInstant + : HoodieTimeline.getCleanRequestedInstant(cleanInstant.getTimestamp())) + .getEarliestInstantToRetain(); + String retainLowerBound; + if (earliestInstantToRetain != null && !StringUtils.isNullOrEmpty(earliestInstantToRetain.getTimestamp())) { + retainLowerBound = earliestInstantToRetain.getTimestamp(); + } else { + // no earliestInstantToRetain, indicate KEEP_LATEST_FILE_VERSIONS clean policy, + // retain first instant after clean instant + retainLowerBound = cleanInstant.getTimestamp(); + } + + oldestInstantToRetain = replaceTimeline.filter(instant -> + HoodieTimeline.compareTimestamps( + instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, - earliestCommitToRetain)) + retainLowerBound)) .firstInstant(); + } else { + oldestInstantToRetain = replaceTimeline.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 Option.empty(); + return oldestInstantToRetain; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java index ce94c92fd4592..b30005b93823b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java @@ -25,17 +25,21 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; /** * Helper class to generate commit metadata. @@ -44,6 +48,7 @@ public class CommitUtils { private static final Logger LOG = LogManager.getLogger(CommitUtils.class); private static final String NULL_SCHEMA_STR = Schema.create(Schema.Type.NULL).toString(); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); /** * Gets the commit action type for given write operation and table type. @@ -117,46 +122,50 @@ private static HoodieCommitMetadata buildMetadataFromStats(List return commitMetadata; } - public static HashMap getFileIdWithoutSuffixAndRelativePathsFromSpecificRecord(Map> - partitionToWriteStats) { - HashMap fileIdToPath = new HashMap<>(); + public static Set> getPartitionAndFileIdWithoutSuffixFromSpecificRecord(Map> + partitionToWriteStats) { + Set> partitionToFileId = new HashSet<>(); // list all partitions paths for (Map.Entry> entry : partitionToWriteStats.entrySet()) { for (org.apache.hudi.avro.model.HoodieWriteStat stat : entry.getValue()) { - fileIdToPath.put(stat.getFileId(), stat.getPath()); + partitionToFileId.add(Pair.of(entry.getKey(), stat.getFileId())); } } - return fileIdToPath; + return partitionToFileId; } - public static HashMap getFileIdWithoutSuffixAndRelativePaths(Map> - partitionToWriteStats) { - HashMap fileIdToPath = new HashMap<>(); + public static Set> getPartitionAndFileIdWithoutSuffix(Map> partitionToWriteStats) { + Set> partitionTofileId = new HashSet<>(); // list all partitions paths for (Map.Entry> entry : partitionToWriteStats.entrySet()) { for (HoodieWriteStat stat : entry.getValue()) { - fileIdToPath.put(stat.getFileId(), stat.getPath()); + partitionTofileId.add(Pair.of(entry.getKey(), stat.getFileId())); } } - return fileIdToPath; + return partitionTofileId; } /** * Process previous commits metadata in the timeline to determine the checkpoint given a checkpoint key. * NOTE: This is very similar in intent to DeltaSync#getLatestCommitMetadataWithValidCheckpointInfo except that - * different deployment models (deltastreamer or spark structured streaming) could have different checkpoint keys. + * different deployment models (deltastreamer or spark structured streaming) could have different checkpoint keys. * - * @param timeline completed commits in active timeline. + * @param timeline completed commits in active timeline. * @param checkpointKey the checkpoint key in the extra metadata of the commit. + * @param keyToLookup key of interest for which checkpoint is looked up for. * @return An optional commit metadata with latest checkpoint. */ - public static Option getLatestCommitMetadataWithValidCheckpointInfo(HoodieTimeline timeline, String checkpointKey) { - return (Option) timeline.filterCompletedInstants().getReverseOrderedInstants().map(instant -> { + public static Option getValidCheckpointForCurrentWriter(HoodieTimeline timeline, String checkpointKey, + String keyToLookup) { + return (Option) timeline.getWriteTimeline().getReverseOrderedInstants().map(instant -> { try { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); - if (StringUtils.nonEmpty(commitMetadata.getMetadata(checkpointKey))) { - return Option.of(commitMetadata); + // process commits only with checkpoint entries + String checkpointValue = commitMetadata.getMetadata(checkpointKey); + if (StringUtils.nonEmpty(checkpointValue)) { + // return if checkpoint for "keyForLookup" exists. + return readCheckpointValue(checkpointValue, keyToLookup); } else { return Option.empty(); } @@ -165,4 +174,27 @@ public static Option getLatestCommitMetadataWithValidCheck } }).filter(Option::isPresent).findFirst().orElse(Option.empty()); } + + public static Option readCheckpointValue(String value, String id) { + try { + Map checkpointMap = OBJECT_MAPPER.readValue(value, Map.class); + if (!checkpointMap.containsKey(id)) { + return Option.empty(); + } + String checkpointVal = checkpointMap.get(id); + return Option.of(checkpointVal); + } catch (IOException e) { + throw new HoodieIOException("Failed to parse checkpoint as map", e); + } + } + + public static String getCheckpointValueAsString(String identifier, String batchId) { + try { + Map checkpointMap = new HashMap<>(); + checkpointMap.put(identifier, batchId); + return OBJECT_MAPPER.writeValueAsString(checkpointMap); + } catch (IOException e) { + throw new HoodieIOException("Failed to parse checkpoint as map", e); + } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java index bc736090d5ebe..d0ef867a2d456 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetUtils.java @@ -37,6 +37,7 @@ import org.apache.parquet.avro.AvroParquetReader; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.column.statistics.Statistics; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; @@ -314,20 +315,25 @@ public List> readRangeFromParquetMetadata( .flatMap(blockMetaData -> blockMetaData.getColumns().stream() .filter(f -> cols.contains(f.getPath().toDotString())) - .map(columnChunkMetaData -> - HoodieColumnRangeMetadata.create( - parquetFilePath.getName(), - columnChunkMetaData.getPath().toDotString(), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - columnChunkMetaData.getStatistics().genericGetMin()), - convertToNativeJavaType( - columnChunkMetaData.getPrimitiveType(), - columnChunkMetaData.getStatistics().genericGetMax()), - columnChunkMetaData.getStatistics().getNumNulls(), - columnChunkMetaData.getValueCount(), - columnChunkMetaData.getTotalSize(), - columnChunkMetaData.getTotalUncompressedSize())) + .map(columnChunkMetaData -> { + Statistics stats = columnChunkMetaData.getStatistics(); + return HoodieColumnRangeMetadata.create( + parquetFilePath.getName(), + columnChunkMetaData.getPath().toDotString(), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + stats.genericGetMin()), + convertToNativeJavaType( + columnChunkMetaData.getPrimitiveType(), + stats.genericGetMax()), + // NOTE: In case when column contains only nulls Parquet won't be creating + // stats for it instead returning stubbed (empty) object. In that case + // we have to equate number of nulls to the value count ourselves + stats.isEmpty() ? columnChunkMetaData.getValueCount() : stats.getNumNulls(), + columnChunkMetaData.getValueCount(), + columnChunkMetaData.getTotalSize(), + columnChunkMetaData.getTotalUncompressedSize()); + }) ) .collect(groupingByCollector); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index d936ee5746579..906757d98df6c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -48,13 +48,15 @@ public class ReflectionUtils { private static final Map> CLAZZ_CACHE = new HashMap<>(); public static Class getClass(String clazzName) { - synchronized (CLAZZ_CACHE) { - if (!CLAZZ_CACHE.containsKey(clazzName)) { - try { - Class clazz = Class.forName(clazzName); - CLAZZ_CACHE.put(clazzName, clazz); - } catch (ClassNotFoundException e) { - throw new HoodieException("Unable to load class", e); + if (!CLAZZ_CACHE.containsKey(clazzName)) { + synchronized (CLAZZ_CACHE) { + if (!CLAZZ_CACHE.containsKey(clazzName)) { + try { + Class clazz = Class.forName(clazzName); + CLAZZ_CACHE.put(clazzName, clazz); + } catch (ClassNotFoundException e) { + throw new HoodieException("Unable to load class", e); + } } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java index d31e648c00e75..6b1069847f3eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java @@ -22,6 +22,8 @@ import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; +import org.apache.avro.generic.GenericData; +import org.apache.hudi.avro.GenericAvroSerializer; import org.apache.avro.util.Utf8; import org.objenesis.strategy.StdInstantiatorStrategy; @@ -122,6 +124,7 @@ public Kryo newKryo() { // Register serializers kryo.register(Utf8.class, new AvroUtf8Serializer()); + kryo.register(GenericData.Fixed.class, new GenericAvroSerializer<>()); return kryo; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java index 9fb0b20e74f2c..e262d8701a2e8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/BitCaskDiskMap.java @@ -443,8 +443,7 @@ private byte[] compressBytes(final byte[] value) throws IOException { private byte[] decompressBytes(final byte[] bytes) throws IOException { decompressBaos.reset(); - InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes)); - try { + try (InputStream in = new InflaterInputStream(new ByteArrayInputStream(bytes))) { int len; while ((len = in.read(decompressIntermediateBuffer)) > 0) { decompressBaos.write(decompressIntermediateBuffer, 0, len); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java index ee930e588d0ed..6b05d69065bf6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/ExternalSpillableMap.java @@ -201,25 +201,20 @@ public R get(Object key) { @Override public R put(T key, R value) { - if (this.currentInMemoryMapSize >= maxInMemorySizeInBytes || inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { - this.estimatedPayloadSize = (long) (this.estimatedPayloadSize * 0.9 - + (keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value)) * 0.1); + if (this.estimatedPayloadSize == 0) { + // At first, use the sizeEstimate of a record being inserted into the spillable map. + // Note, the converter may over-estimate the size of a record in the JVM + this.estimatedPayloadSize = keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); + } else if (this.inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) { + this.estimatedPayloadSize = (long) (this.estimatedPayloadSize * 0.9 + (keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value)) * 0.1); this.currentInMemoryMapSize = this.inMemoryMap.size() * this.estimatedPayloadSize; - LOG.info("Update Estimated Payload size to => " + this.estimatedPayloadSize); } - if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) { - if (estimatedPayloadSize == 0) { - // At first, use the sizeEstimate of a record being inserted into the spillable map. - // Note, the converter may over estimate the size of a record in the JVM - this.estimatedPayloadSize = keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value); - LOG.info("Estimated Payload size => " + estimatedPayloadSize); - } - if (!inMemoryMap.containsKey(key)) { - // TODO : Add support for adjusting payloadSize for updates to the same key - currentInMemoryMapSize += this.estimatedPayloadSize; - } - inMemoryMap.put(key, value); + if (this.inMemoryMap.containsKey(key)) { + this.inMemoryMap.put(key, value); + } else if (this.currentInMemoryMapSize < this.maxInMemorySizeInBytes) { + this.currentInMemoryMapSize += this.estimatedPayloadSize; + this.inMemoryMap.put(key, value); } else { getDiskBasedMap().put(key, value); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatteningIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatteningIterator.java new file mode 100644 index 0000000000000..4adf6b5c91f7c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/FlatteningIterator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util.collection; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * Iterator flattening source {@link Iterator} holding other {@link Iterator}s + */ +public final class FlatteningIterator> implements Iterator { + + private final Iterator sourceIterator; + private Iterator innerSourceIterator; + + public FlatteningIterator(Iterator source) { + this.sourceIterator = source; + } + + public boolean hasNext() { + while (innerSourceIterator == null || !innerSourceIterator.hasNext()) { + if (sourceIterator.hasNext()) { + innerSourceIterator = sourceIterator.next(); + } else { + return false; + } + } + + return true; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + return innerSourceIterator.next(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java index a4655764a97f3..24b0961470bde 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/MappingIterator.java @@ -21,7 +21,9 @@ import java.util.Iterator; import java.util.function.Function; -// TODO java-docs +/** + * Iterator mapping elements of the provided source {@link Iterator} from {@code I} to {@code O} + */ public class MappingIterator implements Iterator { protected final Iterator source; 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 639b5ed4157be..0d85d5a80761a 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/common/util/queue/BoundedInMemoryExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java index c379a7abcc0d9..20e5af1a91cf9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/BoundedInMemoryExecutor.java @@ -61,7 +61,7 @@ protected void doConsume(HoodieMessageQueue queue, HoodieConsumer co } LOG.info("All records from the queue have been consumed"); } catch (Exception e) { - LOG.error("Error consuming records", e); + LOG.error("Failed consuming records", e); queue.markAsFailed(e); throw new HoodieException(e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java index 4ebbd6e528b9a..056c45a0bf40a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorExecutor.java @@ -32,16 +32,23 @@ */ public class DisruptorExecutor extends BaseHoodieQueueBasedExecutor { - public DisruptorExecutor(final Option bufferSize, final Iterator inputItr, - HoodieConsumer consumer, Function transformFunction, Option waitStrategy, Runnable preExecuteRunnable) { + public DisruptorExecutor(Integer bufferSize, + Iterator inputItr, + HoodieConsumer consumer, + Function transformFunction, + String waitStrategy, + Runnable preExecuteRunnable) { this(bufferSize, Collections.singletonList(new IteratorBasedQueueProducer<>(inputItr)), consumer, transformFunction, waitStrategy, preExecuteRunnable); } - public DisruptorExecutor(final Option bufferSize, List> producers, - HoodieConsumer consumer, final Function transformFunction, - final Option waitStrategy, Runnable preExecuteRunnable) { - super(producers, Option.of(consumer), new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategy, producers.size(), preExecuteRunnable), preExecuteRunnable); + public DisruptorExecutor(int bufferSize, + List> producers, + HoodieConsumer consumer, + Function transformFunction, + String waitStrategyId, + Runnable preExecuteRunnable) { + super(producers, Option.of(consumer), new DisruptorMessageQueue<>(bufferSize, transformFunction, waitStrategyId, producers.size(), preExecuteRunnable), preExecuteRunnable); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java index 19b0a04b5e845..6bcdc268d4766 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/DisruptorMessageQueue.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.util.queue; +import com.lmax.disruptor.TimeoutException; import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; @@ -27,7 +28,10 @@ import com.lmax.disruptor.WaitStrategy; import com.lmax.disruptor.dsl.Disruptor; import com.lmax.disruptor.dsl.ProducerType; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import java.util.concurrent.TimeUnit; import java.util.function.Function; /** @@ -38,6 +42,8 @@ */ public class DisruptorMessageQueue implements HoodieMessageQueue { + private static final Logger LOG = LogManager.getLogger(DisruptorMessageQueue.class); + private final Disruptor queue; private final Function transformFunction; private final RingBuffer ringBuffer; @@ -45,11 +51,13 @@ public class DisruptorMessageQueue implements HoodieMessageQueue { private boolean isShutdown = false; private boolean isStarted = false; - public DisruptorMessageQueue(Option bufferSize, Function transformFunction, Option waitStrategyName, int totalProducers, Runnable preExecuteRunnable) { - WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyName); + private static final long TIMEOUT_WAITING_SECS = 10L; + + public DisruptorMessageQueue(int bufferSize, Function transformFunction, String waitStrategyId, int totalProducers, Runnable preExecuteRunnable) { + WaitStrategy waitStrategy = WaitStrategyFactory.build(waitStrategyId); CustomizedThreadFactory threadFactory = new CustomizedThreadFactory("disruptor", true, preExecuteRunnable); - this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize.get(), threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); + this.queue = new Disruptor<>(HoodieDisruptorEvent::new, bufferSize, threadFactory, totalProducers > 1 ? ProducerType.MULTI : ProducerType.SINGLE, waitStrategy); this.ringBuffer = queue.getRingBuffer(); this.transformFunction = transformFunction; } @@ -98,14 +106,30 @@ public void close() { if (!isShutdown) { isShutdown = true; isStarted = false; - queue.shutdown(); + if (Thread.currentThread().isInterrupted()) { + // if current thread has been interrupted, we still give executor a chance to proceeding. + LOG.error("Disruptor Queue has been interrupted! Shutdown now."); + try { + queue.shutdown(TIMEOUT_WAITING_SECS, TimeUnit.SECONDS); + } catch (TimeoutException e) { + LOG.error("Disruptor queue shutdown timeout: " + e); + throw new HoodieException(e); + } + throw new HoodieException("Disruptor Queue has been interrupted! Shutdown now."); + } else { + queue.shutdown(); + } } } } - protected void setHandlers(HoodieConsumer consumer) { + protected void setHandlers(HoodieConsumer consumer) { queue.handleEventsWith((event, sequence, endOfBatch) -> { - consumer.consume(event.get()); + try { + consumer.consume(event.get()); + } catch (Exception e) { + LOG.error("Failed consuming records", e); + } }); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java index 43b2f06103097..770041734239a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/ExecutorType.java @@ -18,12 +18,6 @@ package org.apache.hudi.common.util.queue; -import org.apache.hudi.keygen.constant.KeyGeneratorType; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Types of {@link org.apache.hudi.common.util.queue.HoodieExecutor}. */ @@ -46,12 +40,5 @@ public enum ExecutorType { * The disadvantage is that the executor is a single-write-single-read model, cannot support functions such as speed limit * and can not de-coupe the network read (shuffle read) and network write (writing objects/files to storage) anymore. */ - SIMPLE; - - public static List getNames() { - List names = new ArrayList<>(ExecutorType.values().length); - Arrays.stream(KeyGeneratorType.values()) - .forEach(x -> names.add(x.name())); - return names; - } + SIMPLE } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleHoodieExecutor.java b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleExecutor.java similarity index 53% rename from hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleHoodieExecutor.java rename to hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleExecutor.java index 8d9fcc892bebb..e175c22d93070 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleHoodieExecutor.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/queue/SimpleExecutor.java @@ -18,9 +18,6 @@ package org.apache.hudi.common.util.queue; -import static org.apache.hudi.common.util.ValidationUtils.checkState; - -import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -29,31 +26,31 @@ import java.util.function.Function; /** - * Single Writer and Single Reader mode. Also this SimpleHoodieExecutor has no inner message queue and no inner lock. - * Consuming and writing records from iterator directly. + * Simple implementation of the {@link HoodieExecutor} interface assuming single-writer/single-reader + * mode allowing it to consume from the input {@link Iterator} directly avoiding the need for + * any internal materialization (ie queueing). * - * Compared with queue based Executor - * Advantages: there is no need for additional memory and cpu resources due to lock or multithreading. - * Disadvantages: lost some benefits such as speed limit. And maybe lower throughput. + *

+ * Such executor is aimed primarily at allowing + * the production-consumption chain to run w/ as little overhead as possible, at the expense of + * limited parallelism and therefore throughput, which is not an issue for execution environments + * such as Spark, where it's used primarily in a parallelism constraint environment (on executors) */ -public class SimpleHoodieExecutor implements HoodieExecutor { +public class SimpleExecutor implements HoodieExecutor { - private static final Logger LOG = LogManager.getLogger(SimpleHoodieExecutor.class); + private static final Logger LOG = LogManager.getLogger(SimpleExecutor.class); + // Record iterator (producer) + private final Iterator itr; // Consumer - protected final Option> consumer; - // records iterator - protected final Iterator it; - private final Function transformFunction; + private final HoodieConsumer consumer; - public SimpleHoodieExecutor(final Iterator inputItr, HoodieConsumer consumer, - Function transformFunction) { - this(inputItr, Option.of(consumer), transformFunction); - } + private final Function transformFunction; - public SimpleHoodieExecutor(final Iterator inputItr, Option> consumer, - Function transformFunction) { - this.it = inputItr; + public SimpleExecutor(Iterator inputItr, + HoodieConsumer consumer, + Function transformFunction) { + this.itr = inputItr; this.consumer = consumer; this.transformFunction = transformFunction; } @@ -63,18 +60,16 @@ public SimpleHoodieExecutor(final Iterator inputItr, Option name) { - DisruptorWaitStrategyType strategyType = name.isPresent() ? DisruptorWaitStrategyType.valueOf(name.get().toUpperCase()) : BLOCKING_WAIT; + public static WaitStrategy build(String name) { + DisruptorWaitStrategyType strategyType = DisruptorWaitStrategyType.valueOf(name); switch (strategyType) { case BLOCKING_WAIT: diff --git a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java index 4c926f9f29391..cf66986e155c8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java @@ -278,9 +278,9 @@ public static String createFullName(String name, Deque fieldNames) { public static Map collectRenameCols(InternalSchema oldSchema, InternalSchema newSchema) { List colNamesFromWriteSchema = oldSchema.getAllColsFullName(); return colNamesFromWriteSchema.stream().filter(f -> { - int filedIdFromWriteSchema = oldSchema.findIdByName(f); + int fieldIdFromWriteSchema = oldSchema.findIdByName(f); // try to find the cols which has the same id, but have different colName; - return newSchema.getAllIds().contains(filedIdFromWriteSchema) && !newSchema.findfullName(filedIdFromWriteSchema).equalsIgnoreCase(f); + return newSchema.getAllIds().contains(fieldIdFromWriteSchema) && !newSchema.findfullName(fieldIdFromWriteSchema).equalsIgnoreCase(f); }).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> { int lastDotIndex = e.lastIndexOf("."); return e.substring(lastDotIndex == -1 ? 0 : lastDotIndex + 1); 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 290e3489a3ee8..ff2ca89e98ebc 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 @@ -38,9 +38,9 @@ private SchemaChangeUtils() { /** * Whether to allow the column type to be updated. * now only support: - * int => long/float/double/string - * long => float/double/string - * float => double/String + * int => long/float/double/String/Decimal + * long => float/double/String/Decimal + * float => double/String/Decimal * double => String/Decimal * Decimal => Decimal/String * String => date/decimal @@ -77,6 +77,9 @@ public static boolean isTypeUpdateAllow(Type src, Type dsr) { if (decimalDsr.isWiderThan(decimalSrc)) { return true; } + if (decimalDsr.precision() >= decimalSrc.precision() && decimalDsr.scale() == decimalSrc.scale()) { + return true; + } } else if (dsr.typeId() == Type.TypeID.STRING) { return true; } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroBootstrapFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroBootstrapFileReader.java new file mode 100644 index 0000000000000..62178645ff273 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroBootstrapFileReader.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.util.Option; + +import org.apache.avro.generic.IndexedRecord; + +import java.util.Objects; + +public class HoodieAvroBootstrapFileReader extends HoodieBootstrapFileReader { + + public HoodieAvroBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + super(skeletonFileReader, dataFileReader, partitionFields, partitionValues); + } + + @Override + protected void setPartitionField(int position, Object fieldValue, IndexedRecord row) { + if (Objects.isNull(row.get(position))) { + row.put(position, String.valueOf(fieldValue)); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java index 3834574cd04ca..8edb0dd9f560e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java @@ -18,6 +18,8 @@ package org.apache.hudi.io.storage; +import org.apache.hudi.common.util.Option; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -39,4 +41,9 @@ protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) thr protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { return new HoodieAvroOrcReader(conf, path); } + + @Override + public HoodieFileReader newBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + return new HoodieAvroBootstrapFileReader(skeletonFileReader, dataFileReader, partitionFields, partitionValues); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java index f9a72878b6226..fa6d0fdacf145 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java @@ -102,8 +102,8 @@ public HoodieAvroHFileReader(Configuration hadoopConf, Path path, CacheConfig ca Option.empty()); } - public HoodieAvroHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException { - this(path, HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, hadoopConf), Option.empty()); + public HoodieAvroHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig, FileSystem fs, Option schemaOpt) throws IOException { + this(path, HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, hadoopConf), schemaOpt); } public HoodieAvroHFileReader(FileSystem fs, Path dummyPath, byte[] content, Option schemaOpt) throws IOException { diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java new file mode 100644 index 0000000000000..0de7480789c3d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBootstrapFileReader.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.util.ClosableIterator; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.Schema; + +import java.io.IOException; +import java.util.Set; + +public abstract class HoodieBootstrapFileReader implements HoodieFileReader { + + private final HoodieFileReader skeletonFileReader; + private final HoodieFileReader dataFileReader; + + private final Option partitionFields; + private final Object[] partitionValues; + + public HoodieBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + this.skeletonFileReader = skeletonFileReader; + this.dataFileReader = dataFileReader; + this.partitionFields = partitionFields; + this.partitionValues = partitionValues; + } + + @Override + public String[] readMinMaxRecordKeys() { + return skeletonFileReader.readMinMaxRecordKeys(); + } + + @Override + public BloomFilter readBloomFilter() { + return skeletonFileReader.readBloomFilter(); + } + + @Override + public Set filterRowKeys(Set candidateRowKeys) { + return skeletonFileReader.filterRowKeys(candidateRowKeys); + } + + @Override + public ClosableIterator> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + ClosableIterator> skeletonIterator = skeletonFileReader.getRecordIterator(readerSchema, requestedSchema); + ClosableIterator> dataFileIterator = dataFileReader.getRecordIterator(HoodieAvroUtils.removeMetadataFields(readerSchema), requestedSchema); + return new ClosableIterator>() { + @Override + public void close() { + skeletonIterator.close(); + dataFileIterator.close(); + } + + @Override + public boolean hasNext() { + return skeletonIterator.hasNext() && dataFileIterator.hasNext(); + } + + @Override + public HoodieRecord next() { + HoodieRecord dataRecord = dataFileIterator.next(); + HoodieRecord skeletonRecord = skeletonIterator.next(); + HoodieRecord ret = dataRecord.prependMetaFields(readerSchema, readerSchema, + new MetadataValues().setCommitTime(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.COMMIT_TIME_METADATA_FIELD)) + .setCommitSeqno(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)) + .setRecordKey(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.RECORD_KEY_METADATA_FIELD)) + .setPartitionPath(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.PARTITION_PATH_METADATA_FIELD)) + .setFileName(skeletonRecord.getRecordKey(readerSchema, HoodieRecord.FILENAME_METADATA_FIELD)), null); + if (partitionFields.isPresent()) { + for (int i = 0; i < partitionValues.length; i++) { + int position = readerSchema.getField(partitionFields.get()[i]).pos(); + setPartitionField(position, partitionValues[i], ret.getData()); + } + } + return ret; + } + }; + } + + protected abstract void setPartitionField(int position, Object fieldValue, T row); + + @Override + public Schema getSchema() { + // return merged schema (meta fields + data file schema) + return HoodieAvroUtils.addMetadataFields(dataFileReader.getSchema()); + } + + @Override + public void close() { + skeletonFileReader.close(); + dataFileReader.close(); + } + + @Override + public long getTotalRecords() { + return Math.min(skeletonFileReader.getTotalRecords(), dataFileReader.getTotalRecords()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java index 36b3f3a57a31a..5fe797f9797ff 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java @@ -18,15 +18,16 @@ package org.apache.hudi.io.storage; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; - import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; + import java.io.IOException; import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; @@ -83,4 +84,8 @@ protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) thr protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) { throw new UnsupportedOperationException(); } + + public HoodieFileReader newBootstrapFileReader(HoodieFileReader skeletonFileReader, HoodieFileReader dataFileReader, Option partitionFields, Object[] partitionValues) { + throw new UnsupportedOperationException(); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 1034aa3125532..f343d6437cfa0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -19,12 +19,12 @@ package org.apache.hudi.metadata; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.bloom.BloomFilterFactory; import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -73,11 +73,12 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class); - public static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; - public static final int BUFFER_SIZE = 10 * 1024 * 1024; + protected static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024; + // NOTE: Buffer-size is deliberately set pretty low, since MT internally is relying + // on HFile (serving as persisted binary key-value mapping) to do caching + protected static final int BUFFER_SIZE = 10 * 1024; // 10Kb protected final transient HoodieEngineContext engineContext; - protected final SerializableConfiguration hadoopConf; protected final SerializablePath dataBasePath; protected final HoodieTableMetaClient dataMetaClient; protected final Option metrics; @@ -92,9 +93,11 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata { protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String dataBasePath, String spillableMapDirectory) { this.engineContext = engineContext; - this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf()); this.dataBasePath = new SerializablePath(new CachingPath(dataBasePath)); - this.dataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(dataBasePath).build(); + this.dataMetaClient = HoodieTableMetaClient.builder() + .setConf(engineContext.getHadoopConf().get()) + .setBasePath(dataBasePath) + .build(); this.spillableMapDirectory = spillableMapDirectory; this.metadataConfig = metadataConfig; @@ -123,8 +126,10 @@ public List getAllPartitionPaths() throws IOException { throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e); } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath.toString(), - metadataConfig.shouldAssumeDatePartitioning()).getAllPartitionPaths(); + + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + createFileSystemBackedTableMetadata(); + return fileSystemBackedTableMetadata.getAllPartitionPaths(); } /** @@ -148,8 +153,9 @@ public FileStatus[] getAllFilesInPartition(Path partitionPath) } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath.toString(), metadataConfig.shouldAssumeDatePartitioning()) - .getAllFilesInPartition(partitionPath); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + createFileSystemBackedTableMetadata(); + return fileSystemBackedTableMetadata.getAllFilesInPartition(partitionPath); } @Override @@ -168,8 +174,9 @@ public Map getAllFilesInPartitions(Collection part } } - return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, dataBasePath.toString(), metadataConfig.shouldAssumeDatePartitioning()) - .getAllFilesInPartitions(partitions); + FileSystemBackedTableMetadata fileSystemBackedTableMetadata = + createFileSystemBackedTableMetadata(); + return fileSystemBackedTableMetadata.getAllFilesInPartitions(partitions); } @Override @@ -205,6 +212,7 @@ public Map, BloomFilter> getBloomFilters(final List partitionIDFileIDSortedStrings = new TreeSet<>(); Map> fileToKeyMap = new HashMap<>(); + // TODO simplify (no sorting is required) partitionNameFileNameList.forEach(partitionNameFileNamePair -> { final String bloomFilterIndexKey = HoodieMetadataPayload.getBloomFilterIndexKey( new PartitionIndexID(partitionNameFileNamePair.getLeft()), new FileIndexID(partitionNameFileNamePair.getRight())); @@ -227,7 +235,11 @@ public Map, BloomFilter> getBloomFilters(final List fetchAllFilesInPartitionPaths(List partitionPath getRecordsByKeys(new ArrayList<>(partitionIdToPathMap.keySet()), MetadataPartitionType.FILES.getPartitionPath()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer())); - FileSystem fs = partitionPaths.get(0).getFileSystem(hadoopConf.get()); + FileSystem fs = partitionPaths.get(0).getFileSystem(getHadoopConf()); Map partitionPathToFilesMap = partitionIdRecordPairs.parallelStream() .map(pair -> { @@ -399,18 +411,27 @@ private void checkForSpuriousDeletes(HoodieMetadataPayload metadataPayload, Stri } } + private FileSystemBackedTableMetadata createFileSystemBackedTableMetadata() { + return new FileSystemBackedTableMetadata(getEngineContext(), dataMetaClient.getSerializableHadoopConf(), dataBasePath.toString(), + metadataConfig.shouldAssumeDatePartitioning()); + } + protected abstract Option> getRecordByKey(String key, String partitionName); public abstract List>>> getRecordsByKeys(List key, String partitionName); protected HoodieEngineContext getEngineContext() { - return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get()); + return engineContext != null ? engineContext : new HoodieLocalEngineContext(getHadoopConf()); } public HoodieMetadataConfig getMetadataConfig() { return metadataConfig; } + protected Configuration getHadoopConf() { + return dataMetaClient.getHadoopConf(); + } + protected String getLatestDataInstantTime() { return dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); 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 09c6b35309a3a..f3538127955f8 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,9 +22,9 @@ 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; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodieListData; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -55,6 +55,7 @@ import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.io.storage.HoodieFileReaderFactory; import org.apache.hudi.io.storage.HoodieSeekingFileReader; +import org.apache.hudi.util.Transient; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -82,6 +83,7 @@ import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_FILES; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getFileSystemView; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit; /** * Table metadata provided by an internal DFS backed Hudi metadata table. @@ -90,19 +92,18 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class); - private static final Schema METADATA_RECORD_SCHEMA = HoodieMetadataRecord.getClassSchema(); + private final String metadataBasePath; - private String metadataBasePath; - // Metadata table's timeline and metaclient private HoodieTableMetaClient metadataMetaClient; private HoodieTableConfig metadataTableConfig; + private HoodieTableFileSystemView metadataFileSystemView; // should we reuse the open file handles, across calls private final boolean reuse; // Readers for the latest file slice corresponding to file groups in the metadata partition - private final Map, Pair, HoodieMetadataLogRecordReader>> partitionReaders = - new ConcurrentHashMap<>(); + private final Transient, Pair, HoodieMetadataLogRecordReader>>> partitionReaders = + Transient.lazy(ConcurrentHashMap::new); public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath, String spillableMapDirectory) { @@ -113,18 +114,19 @@ public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetada String datasetBasePath, String spillableMapDirectory, boolean reuse) { super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory); this.reuse = reuse; + this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath.toString()); + initIfNeeded(); } private void initIfNeeded() { - this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(dataBasePath.toString()); if (!isMetadataTableEnabled) { if (!HoodieTableMetadata.isMetadataTable(metadataBasePath)) { LOG.info("Metadata table is disabled."); } } else if (this.metadataMetaClient == null) { try { - this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()).setBasePath(metadataBasePath).build(); + this.metadataMetaClient = HoodieTableMetaClient.builder().setConf(getHadoopConf()).setBasePath(metadataBasePath).build(); this.metadataFileSystemView = getFileSystemView(metadataMetaClient); this.metadataTableConfig = metadataMetaClient.getTableConfig(); this.isBloomFilterIndexEnabled = metadataConfig.isBloomFilterIndexEnabled(); @@ -213,14 +215,14 @@ public HoodieData> getRecordsByKeyPrefixes(L return mergedRecords.stream() .map(keyRecordPair -> keyRecordPair.getValue().orElse(null)) + .filter(Objects::nonNull) .iterator(); } catch (IOException ioe) { throw new HoodieIOException("Error merging records from metadata table for " + sortedKeyPrefixes.size() + " key : ", ioe); } finally { closeReader(readers); } - }) - .filter(Objects::nonNull); + }); } @Override @@ -425,7 +427,7 @@ private Map, List> getPartitionFileSliceToKeysMa private Pair, HoodieMetadataLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) { if (reuse) { Pair key = Pair.of(partitionName, slice.getFileId()); - return partitionReaders.computeIfAbsent(key, ignored -> openReaders(partitionName, slice)); + return partitionReaders.get().computeIfAbsent(key, ignored -> openReaders(partitionName, slice)); } else { return openReaders(partitionName, slice); } @@ -462,7 +464,7 @@ private Pair, Long> getBaseFileReader(FileSlice slice if (basefile.isPresent()) { String baseFilePath = basefile.get().getPath(); baseFileReader = (HoodieSeekingFileReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO) - .getFileReader(hadoopConf.get(), new Path(baseFilePath)); + .getFileReader(getHadoopConf(), new Path(baseFilePath)); baseFileOpenMs = timer.endTimer(); LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", baseFilePath, basefile.get().getCommitTime(), baseFileOpenMs)); @@ -481,6 +483,15 @@ private Set getValidInstantTimestamps() { Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstantsAsStream() .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + // We should also add completed indexing delta commits in the metadata table, as they do not + // have corresponding completed instant in the data table + validInstantTimestamps.addAll( + metadataMetaClient.getActiveTimeline() + .filter(instant -> instant.isCompleted() && isIndexingCommit(instant.getTimestamp())) + .getInstants().stream() + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList())); + // For any rollbacks and restores, we cannot neglect the instants that they are rolling back. // The rollback instant should be more recent than the start of the timeline for it to have rolled back any // instant which we have a log block for. @@ -531,7 +542,7 @@ public Pair getLogRecordScanner(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<>(); @@ -596,7 +618,7 @@ public void close() { */ private synchronized void close(Pair partitionFileSlicePair) { Pair, HoodieMetadataLogRecordReader> readers = - partitionReaders.remove(partitionFileSlicePair); + partitionReaders.get().remove(partitionFileSlicePair); closeReader(readers); } @@ -604,10 +626,10 @@ private synchronized void close(Pair partitionFileSlicePair) { * Close and clear all the partitions readers. */ private void closePartitionReaders() { - for (Pair partitionFileSlicePair : partitionReaders.keySet()) { + for (Pair partitionFileSlicePair : partitionReaders.get().keySet()) { close(partitionFileSlicePair); } - partitionReaders.clear(); + partitionReaders.get().clear(); } private void closeReader(Pair, HoodieMetadataLogRecordReader> readers) { @@ -629,10 +651,6 @@ public boolean enabled() { return isMetadataTableEnabled; } - public SerializableConfiguration getHadoopConf() { - return hadoopConf; - } - public HoodieTableMetaClient getMetadataMetaClient() { return metadataMetaClient; } 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 76b341609ded9..a5e25d6130c42 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 @@ -54,9 +54,8 @@ public HoodieMetadataFileSystemView(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, HoodieMetadataConfig metadataConfig) { - super(metaClient, visibleActiveTimeline); - this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(), - FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue(), true); + this(metaClient, visibleActiveTimeline, HoodieTableMetadata.create(engineContext, metadataConfig, + metaClient.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue(), true)); } /** @@ -92,14 +91,26 @@ protected Map, FileStatus[]> listPartitions(List convertMetadataToFilesPartitionRecords(HoodieCo // Add record bearing added partitions list List partitionsAdded = getPartitionsAdded(commitMetadata); - // Add record bearing deleted partitions list - List partitionsDeleted = getPartitionsDeleted(commitMetadata); - - records.add(HoodieMetadataPayload.createPartitionListRecord(partitionsAdded, partitionsDeleted)); + records.add(HoodieMetadataPayload.createPartitionListRecord(partitionsAdded)); // Update files listing records for each individual partition List> updatedPartitionFilesRecords = @@ -374,21 +375,6 @@ private static List getPartitionsAdded(HoodieCommitMetadata commitMetada .collect(Collectors.toList()); } - private static List getPartitionsDeleted(HoodieCommitMetadata commitMetadata) { - if (commitMetadata instanceof HoodieReplaceCommitMetadata - && WriteOperationType.DELETE_PARTITION.equals(commitMetadata.getOperationType())) { - Map> partitionToReplaceFileIds = - ((HoodieReplaceCommitMetadata) commitMetadata).getPartitionToReplaceFileIds(); - - return partitionToReplaceFileIds.keySet().stream() - // We need to make sure we properly handle case of non-partitioned tables - .map(HoodieTableMetadataUtil::getPartitionIdentifier) - .collect(Collectors.toList()); - } - - return Collections.emptyList(); - } - /** * Convert commit action metadata to bloom filter records. * @@ -1380,4 +1366,18 @@ public static Set getInflightAndCompletedMetadataPartitions(HoodieTableC inflightAndCompletedPartitions.addAll(tableConfig.getMetadataPartitions()); return inflightAndCompletedPartitions; } + + /** + * Checks if a delta commit in metadata table is written by async indexer. + *

+ * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks in the + * metadata table is landed. + * + * @param instantTime Instant time to check. + * @return {@code true} if from async indexer; {@code false} otherwise. + */ + public static boolean isIndexingCommit(String instantTime) { + return instantTime.length() == MILLIS_INSTANT_ID_LENGTH + METADATA_INDEXER_TIME_SUFFIX.length() + && instantTime.endsWith(METADATA_INDEXER_TIME_SUFFIX); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/util/Transient.java b/hudi-common/src/main/java/org/apache/hudi/util/Transient.java new file mode 100644 index 0000000000000..0d8f6ad6565e5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/util/Transient.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.util; + +import org.apache.hudi.common.function.SerializableSupplier; +import org.apache.hudi.common.function.ThrowingConsumer; + +import javax.annotation.concurrent.ThreadSafe; +import java.io.Serializable; + +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; + +/** + * {@link Serializable} counterpart of {@link Lazy} + * + * @param type of the object being held by {@link Transient} + */ +@ThreadSafe +public class Transient implements Serializable { + + private SerializableSupplier initializer; + + private transient boolean initialized; + private transient T ref; + + private Transient(SerializableSupplier initializer) { + checkArgument(initializer != null); + + this.initializer = initializer; + this.ref = null; + this.initialized = false; + } + + private Transient(T value, SerializableSupplier initializer) { + checkArgument(value != null); + checkArgument(initializer != null); + + this.initializer = initializer; + this.ref = value; + this.initialized = true; + } + + public T get() { + if (!initialized) { + synchronized (this) { + if (!initialized) { + this.ref = initializer.get(); + initialized = true; + } + } + } + + return ref; + } + + public void reset() { + synchronized (this) { + this.ref = null; + this.initialized = false; + } + } + + public void destroy(ThrowingConsumer cleaner) throws Exception { + synchronized (this) { + if (initialized) { + cleaner.accept(ref); + } + + this.ref = null; + this.initialized = false; + this.initializer = null; + } + } + + /** + * Creates instance of {@link Transient} by lazily executing provided {@code initializer}, + * to instantiate value of type {@link T}. Same initializer will be used to re-instantiate + * the value after original one being dropped during serialization/deserialization cycle + */ + public static Transient lazy(SerializableSupplier initializer) { + return new Transient<>(initializer); + } + + /** + * Creates instance of {@link Transient} by eagerly setting it to provided {@code value}, + * while given {@code initializer} will be used to re-instantiate the value after original + * one being dropped during serialization/deserialization cycle + */ + public static Transient eager(T value, SerializableSupplier initializer) { + return new Transient<>(value, initializer); + } +} \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java index 14d65e8359a2c..c05683e605cdb 100644 --- a/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/avro/TestAvroSchemaUtils.java @@ -18,10 +18,16 @@ package org.apache.hudi.avro; +import org.apache.hudi.exception.SchemaCompatibilityException; + import org.apache.avro.Schema; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import java.util.Collections; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestAvroSchemaUtils { @@ -172,4 +178,55 @@ public void testIsCompatibleProjection() { Schema.createUnion(Schema.create(Schema.Type.NULL), sourceSchema), Schema.createUnion(Schema.create(Schema.Type.NULL), projectedNestedSchema))); } + + private static final Schema FULL_SCHEMA = new Schema.Parser().parse("{\n" + + " \"type\" : \"record\",\n" + + " \"name\" : \"record\",\n" + + " \"fields\" : [ {\n" + + " \"name\" : \"a\",\n" + + " \"type\" : [ \"null\", \"int\" ],\n" + + " \"default\" : null\n" + + " }, {\n" + + " \"name\" : \"b\",\n" + + " \"type\" : [ \"null\", \"int\" ],\n" + + " \"default\" : null\n" + + " }, {\n" + + " \"name\" : \"c\",\n" + + " \"type\" : [ \"null\", \"int\" ],\n" + + " \"default\" : null\n" + + " } ]\n" + + "}"); + + private static final Schema SHORT_SCHEMA = new Schema.Parser().parse("{\n" + + " \"type\" : \"record\",\n" + + " \"name\" : \"record\",\n" + + " \"fields\" : [ {\n" + + " \"name\" : \"a\",\n" + + " \"type\" : [ \"null\", \"int\" ],\n" + + " \"default\" : null\n" + + " }, {\n" + + " \"name\" : \"b\",\n" + + " \"type\" : [ \"null\", \"int\" ],\n" + + " \"default\" : null\n" + + " } ]\n" + + "}\n"); + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testIsCompatibleProjectionNotAllowed(boolean shouldValidate) { + assertThrows(SchemaCompatibilityException.class, + () -> AvroSchemaUtils.checkSchemaCompatible(FULL_SCHEMA, SHORT_SCHEMA, shouldValidate, false, Collections.emptySet())); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testIsCompatibleProjectionAllowed(boolean shouldValidate) { + AvroSchemaUtils.checkSchemaCompatible(FULL_SCHEMA, SHORT_SCHEMA, shouldValidate, true, Collections.emptySet()); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testIsCompatiblePartitionDropCols(boolean shouldValidate) { + AvroSchemaUtils.checkSchemaCompatible(FULL_SCHEMA, SHORT_SCHEMA, shouldValidate, false, Collections.singleton("c")); + } } 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 149783ba4166b..b4052750fa533 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.List; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; /** @@ -87,6 +88,15 @@ public void testGetSchema() { assertDoesNotThrow(fs::getScheme, "Method #getSchema does not implement correctly"); } + @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. */ @@ -95,6 +105,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; @@ -218,5 +229,16 @@ public Configuration getConf() { public String getScheme() { return fs.getScheme(); } + + @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/fs/TestStorageSchemes.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java index 7f2e0c2f8de28..7ac8a9bcabb63 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestStorageSchemes.java @@ -54,6 +54,28 @@ public void testStorageSchemes() { assertFalse(StorageSchemes.isAppendSupported("oci")); assertFalse(StorageSchemes.isAppendSupported("tos")); assertTrue(StorageSchemes.isAppendSupported("cfs")); + + assertTrue(StorageSchemes.isAtomicCreationSupported("file")); + assertTrue(StorageSchemes.isAtomicCreationSupported("hdfs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("afs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("s3a")); + assertFalse(StorageSchemes.isAtomicCreationSupported("gs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("wasb")); + assertFalse(StorageSchemes.isAtomicCreationSupported("adl")); + assertFalse(StorageSchemes.isAtomicCreationSupported("abfs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("oss")); + assertTrue(StorageSchemes.isAtomicCreationSupported("viewfs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("alluxio")); + assertFalse(StorageSchemes.isAtomicCreationSupported("cosn")); + assertFalse(StorageSchemes.isAtomicCreationSupported("dbfs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("cos")); + assertFalse(StorageSchemes.isAtomicCreationSupported("jfs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("bos")); + assertFalse(StorageSchemes.isAtomicCreationSupported("ks3")); + assertFalse(StorageSchemes.isAtomicCreationSupported("ofs")); + assertFalse(StorageSchemes.isAtomicCreationSupported("oci")); + assertFalse(StorageSchemes.isAtomicCreationSupported("tos")); + assertFalse(StorageSchemes.isAtomicCreationSupported("cfs")); assertThrows(IllegalArgumentException.class, () -> { StorageSchemes.isAppendSupported("s2"); }, "Should throw exception for unsupported schemes"); 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 a8828514eebe9..8689a2b617899 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 @@ -551,7 +551,7 @@ public void testCDCBlock() throws IOException, InterruptedException { + "]}"; Schema dataSchema = new Schema.Parser().parse(dataSchameString); Schema cdcSchema = HoodieCDCUtils.schemaBySupplementalLoggingMode( - HoodieCDCSupplementalLoggingMode.WITH_BEFORE_AFTER, dataSchema); + HoodieCDCSupplementalLoggingMode.data_before_after, dataSchema); GenericRecord insertedRecord = new GenericData.Record(dataSchema); insertedRecord.put("uuid", 1); insertedRecord.put("name", "apple"); @@ -625,7 +625,7 @@ public void testCDCBlock() throws IOException, InterruptedException { public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Generate 4 delta-log files w/ random records @@ -652,7 +652,7 @@ public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); @@ -672,7 +672,7 @@ public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType public void testBasicAppendAndPartialScanning(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanV2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Generate 3 delta-log files w/ random records Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); @@ -692,14 +692,14 @@ public void testBasicAppendAndPartialScanning(ExternalSpillableMap.DiskMapType d .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList())) .withReaderSchema(schema) .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(1024L) + .withMaxMemorySizeInBytes(10240L) .withReadBlocksLazily(readBlocksLazily) .withReverseReader(false) .withBufferSize(BUFFER_SIZE) .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanV2) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .withForceFullScan(false) .build(); @@ -763,7 +763,7 @@ public void testBasicAppendAndPartialScanning(ExternalSpillableMap.DiskMapType d public void testBasicAppendAndPartialScanningByKeyPrefixes(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanV2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Generate 3 delta-log files w/ random records Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); @@ -783,14 +783,14 @@ public void testBasicAppendAndPartialScanningByKeyPrefixes(ExternalSpillableMap. .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList())) .withReaderSchema(schema) .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(1024L) + .withMaxMemorySizeInBytes(10240L) .withReadBlocksLazily(readBlocksLazily) .withReverseReader(false) .withBufferSize(BUFFER_SIZE) .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanV2) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .withForceFullScan(false) .build(); @@ -1049,7 +1049,7 @@ public void testValidateCorruptBlockEndPosition() throws IOException, URISyntaxE public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1078,38 +1078,14 @@ public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMa writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - FileCreateUtils.createDeltaCommit(basePath, "100", fs); - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(200, scanner.getTotalLogRecords()); - Set readKeys = new HashSet<>(200); - scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(200, readKeys.size(), "Stream collect should return all 200 records"); copyOfRecords1.addAll(copyOfRecords2); Set originalKeys = copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect(Collectors.toSet()); - assertEquals(originalKeys, readKeys, "CompositeAvroLogReader should return 200 records from 2 versions"); - scanner.close(); + checkLogBlocksAndKeys("100", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 200, 200, Option.of(originalKeys)); } @ParameterizedTest @@ -1117,7 +1093,7 @@ public void testAvroLogRecordReaderBasic(ExternalSpillableMap.DiskMapType diskMa public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1165,45 +1141,22 @@ public void testAvroLogRecordReaderWithRollbackTombstone(ExternalSpillableMap.Di writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - FileCreateUtils.createDeltaCommit(basePath, "100", fs); FileCreateUtils.createDeltaCommit(basePath, "102", fs); - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("102") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches"); - Set readKeys = new HashSet<>(200); - scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(200, readKeys.size(), "Stream collect should return all 200 records"); + copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect(Collectors.toSet()); - assertEquals(originalKeys, readKeys, "CompositeAvroLogReader should return 200 records from 2 versions"); - scanner.close(); + checkLogBlocksAndKeys("102", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 200, 200, Option.of(originalKeys)); } @ParameterizedTest @MethodSource("testArguments") public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1257,38 +1210,15 @@ public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.D writer.appendBlock(dataBlock); writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - FileCreateUtils.createDeltaCommit(basePath, "100", fs); FileCreateUtils.createDeltaCommit(basePath, "103", fs); - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("103") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(true) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records"); - Set readKeys = new HashSet<>(200); - scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(200, readKeys.size(), "Stream collect should return all 200 records"); + copyOfRecords1.addAll(copyOfRecords3); Set originalKeys = copyOfRecords1.stream().map(s -> ((GenericRecord) s).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString()) .collect(Collectors.toSet()); - assertEquals(originalKeys, readKeys, "CompositeAvroLogReader should return 200 records from 2 versions"); - scanner.close(); + checkLogBlocksAndKeys("103", schema, true, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 200, 200, Option.of(originalKeys)); } @ParameterizedTest @@ -1296,7 +1226,7 @@ public void testAvroLogRecordReaderWithFailedPartialBlock(ExternalSpillableMap.D public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1359,7 +1289,7 @@ public void testAvroLogRecordReaderWithDeleteAndRollback(ExternalSpillableMap.Di .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); @@ -1560,7 +1490,7 @@ public void testAvroLogRecordReaderWithDisorderDelete(ExternalSpillableMap.DiskM public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Write a Data block and Delete block with same InstantTime (written in same batch) @@ -1614,36 +1544,11 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk } // Attempt 2 : Write another rollback blocks for a failed write writer.appendBlock(commandBlock); + writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - - // all data must be rolled back before merge - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback"); - - final List readKeys = new ArrayList<>(); - scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(0, readKeys.size(), "Stream collect should return all 0 records"); + checkLogBlocksAndKeys("100", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 0, 0, Option.empty()); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); - writer.close(); - scanner.close(); } @ParameterizedTest @@ -1651,7 +1556,7 @@ public void testAvroLogRecordReaderWithFailedRollbacks(ExternalSpillableMap.Disk public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Write a Data block and Delete block with same InstantTime (written in same batch) @@ -1689,31 +1594,11 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); writer.appendBlock(commandBlock); + writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); + checkLogBlocksAndKeys("100", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 0, 0, Option.empty()); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); - writer.close(); - scanner.close(); } @ParameterizedTest @@ -1721,7 +1606,7 @@ public void testAvroLogRecordReaderWithInsertDeleteAndRollback(ExternalSpillable public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); // Set a small threshold so that every block is a new version @@ -1746,33 +1631,10 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); - - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("100") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records"); - final List readKeys = new ArrayList<>(100); - scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); - assertEquals(100, readKeys.size(), "Stream collect should return all 150 records"); writer.close(); - scanner.close(); + + checkLogBlocksAndKeys("100", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 100, 100, Option.empty()); } @ParameterizedTest @@ -1780,7 +1642,7 @@ public void testAvroLogRecordReaderWithInvalidRollback(ExternalSpillableMap.Disk public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Write a 3 Data blocs with same InstantTime (written in same batch) @@ -1822,30 +1684,65 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); writer.appendBlock(commandBlock); + writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); + checkLogBlocksAndKeys("101", schema, readBlocksLazily, diskMapType, isCompressionEnabled, enableOptimizedLogBlocksScan, + 0, 0, Option.empty()); + } - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("101") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); + @Test + public void testAvroLogRecordReaderWithRollbackOlderBlocks() + throws IOException, URISyntaxException, InterruptedException { + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + // Set a small threshold so that every block is a new version + Writer writer = + HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION) + .withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build(); + + // Write 1 + SchemaTestUtil testUtil = new SchemaTestUtil(); + List records1 = testUtil.generateHoodieTestRecords(0, 100); + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + HoodieDataBlock dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records1, header); + writer.appendBlock(dataBlock); + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + + // Write 2 + List records2 = testUtil.generateHoodieTestRecords(100, 10); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString()); + dataBlock = getDataBlock(DEFAULT_DATA_BLOCK_TYPE, records2, header); + writer.appendBlock(dataBlock); + FileCreateUtils.createDeltaCommit(basePath, "101", fs); + + // Should be able to read all 110 records + checkLogBlocksAndKeys("101", schema, true, ExternalSpillableMap.DiskMapType.BITCASK, false, + false, 110, 110, Option.empty()); + + // Write a rollback for commit 100 which is not the latest commit + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); + HoodieCommandBlock commandBlock = new HoodieCommandBlock(header); + writer.appendBlock(commandBlock); + + // Should only be able to read 10 records from commit 101 + checkLogBlocksAndKeys("101", schema, true, ExternalSpillableMap.DiskMapType.BITCASK, false, + false, 10, 10, Option.empty()); + + // Write a rollback for commit 101 which is the latest commit + header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "101"); + header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, + String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); + commandBlock = new HoodieCommandBlock(header); + writer.appendBlock(commandBlock); writer.close(); - scanner.close(); + + // Should not read any records as both commits are rolled back + checkLogBlocksAndKeys("101", schema, true, ExternalSpillableMap.DiskMapType.BITCASK, false, + false, 0, 0, Option.empty()); } @ParameterizedTest @@ -1853,7 +1750,7 @@ public void testAvroLogRecordReaderWithInsertsDeleteAndRollback(ExternalSpillabl public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) + boolean enableOptimizedLogBlocksScan) throws IOException, URISyntaxException, InterruptedException { // Write a 3 Data blocs with same InstantTime (written in same batch) @@ -1935,33 +1832,13 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsAndRollback(ExternalS writer.appendBlock(commandBlock); writer.close(); - List allLogFiles = - FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") - .map(s -> s.getPath().toString()).collect(Collectors.toList()); - - HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() - .withFileSystem(fs) - .withBasePath(basePath) - .withLogFilePaths(allLogFiles) - .withReaderSchema(schema) - .withLatestInstantTime("101") - .withMaxMemorySizeInBytes(10240L) - .withReadBlocksLazily(readBlocksLazily) - .withReverseReader(false) - .withBufferSize(BUFFER_SIZE) - .withSpillableMapBasePath(spillableBasePath) - .withDiskMapType(diskMapType) - .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) - .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) - .build(); - assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records"); + checkLogBlocksAndKeys("101", schema, true, ExternalSpillableMap.DiskMapType.BITCASK, false, + false, 0, 0, Option.empty()); FileCreateUtils.deleteDeltaCommit(basePath, "100", fs); - scanner.close(); } @ParameterizedTest - @MethodSource("testArgumentsWithoutScanV2Arg") + @MethodSource("testArgumentsWithoutOptimizedScanArg") public void testAvroLogRecordReaderWithMixedInsertsCorruptsRollbackAndMergedLogBlock(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily) @@ -2134,7 +2011,7 @@ public void testAvroLogRecordReaderWithMixedInsertsCorruptsRollbackAndMergedLogB .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(true) + .withOptimizedLogBlocksScan(true) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); assertEquals(600, scanner.getTotalLogRecords(), "We would read 600 records from scanner"); @@ -2170,7 +2047,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) { + boolean enableOptimizedLogBlocksScan) { try { // Write one Data block with same InstantTime (written in same batch) Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); @@ -2224,7 +2101,7 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 .withSpillableMapBasePath(spillableBasePath) .withDiskMapType(diskMapType) .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) - .withUseScanV2(useScanv2) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); @@ -2241,13 +2118,13 @@ private void testAvroLogRecordReaderMergingMultipleLogFiles(int numRecordsInLog1 public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) { + boolean enableOptimizedLogBlocksScan) { /* * FIRST_ATTEMPT_FAILED: * Original task from the stage attempt failed, but subsequent stage retry succeeded. */ testAvroLogRecordReaderMergingMultipleLogFiles(77, 100, - diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); + diskMapType, isCompressionEnabled, readBlocksLazily, enableOptimizedLogBlocksScan); } @ParameterizedTest @@ -2255,13 +2132,13 @@ public void testAvroLogRecordReaderWithFailedTaskInFirstStageAttempt(ExternalSpi public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) { + boolean enableOptimizedLogBlocksScan) { /* * SECOND_ATTEMPT_FAILED: * Original task from stage attempt succeeded, but subsequent retry attempt failed. */ testAvroLogRecordReaderMergingMultipleLogFiles(100, 66, - diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); + diskMapType, isCompressionEnabled, readBlocksLazily, enableOptimizedLogBlocksScan); } @ParameterizedTest @@ -2269,13 +2146,13 @@ public void testAvroLogRecordReaderWithFailedTaskInSecondStageAttempt(ExternalSp public void testAvroLogRecordReaderTasksSucceededInBothStageAttempts(ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean readBlocksLazily, - boolean useScanv2) { + boolean enableOptimizedLogBlocksScan) { /* * BOTH_ATTEMPTS_SUCCEEDED: * Original task from the stage attempt and duplicate task from the stage retry succeeded. */ testAvroLogRecordReaderMergingMultipleLogFiles(100, 100, - diskMapType, isCompressionEnabled, readBlocksLazily, useScanv2); + diskMapType, isCompressionEnabled, readBlocksLazily, enableOptimizedLogBlocksScan); } @ParameterizedTest @@ -2593,7 +2470,7 @@ private static HoodieDataBlock getDataBlock(HoodieLogBlockType dataBlockType, Li } private static Stream testArguments() { - // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily, Arg4: useScanv2 + // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily, Arg4: enableOptimizedLogBlocksScan return Stream.of( arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false, true), arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false, true), @@ -2614,8 +2491,8 @@ private static Stream testArguments() { ); } - private static Stream testArgumentsWithoutScanV2Arg() { - // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily, Arg4: useScanv2 + private static Stream testArgumentsWithoutOptimizedScanArg() { + // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: readBlocksLazily return Stream.of( arguments(ExternalSpillableMap.DiskMapType.BITCASK, false, false), arguments(ExternalSpillableMap.DiskMapType.ROCKS_DB, false, false), @@ -2718,4 +2595,38 @@ private HoodieLogFormat.Reader createCorruptedFile(String fileId) throws Excepti return reader; } + + private void checkLogBlocksAndKeys(String latestInstantTime, Schema schema, boolean readBlocksLazily, + ExternalSpillableMap.DiskMapType diskMapType, boolean isCompressionEnabled, boolean enableOptimizedLogBlocksScan, int expectedTotalRecords, + int expectedTotalKeys, Option> expectedKeys) throws IOException { + List allLogFiles = + FSUtils.getAllLogFiles(fs, partitionPath, "test-fileid1", HoodieLogFile.DELTA_EXTENSION, "100") + .map(s -> s.getPath().toString()).collect(Collectors.toList()); + + HoodieMergedLogRecordScanner.Builder builder = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths(allLogFiles) + .withReaderSchema(schema) + .withLatestInstantTime(latestInstantTime) + .withMaxMemorySizeInBytes(10240L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(BUFFER_SIZE) + .withSpillableMapBasePath(spillableBasePath) + .withDiskMapType(diskMapType) + .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withOptimizedLogBlocksScan(enableOptimizedLogBlocksScan) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())); + try (HoodieMergedLogRecordScanner scanner = builder.build()) { + assertEquals(expectedTotalRecords, scanner.getTotalLogRecords(), "There should be " + expectedTotalRecords + " records"); + final Set readKeys = new HashSet<>(); + scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey())); + assertEquals(expectedTotalKeys, readKeys.size(), "Read should return return all " + expectedTotalKeys + " keys"); + + if (expectedKeys.isPresent()) { + assertEquals(expectedKeys.get(), readKeys, "Keys read from log file should match written keys"); + } + } + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java index 2d0fd5b094e62..9f578e27a462c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java @@ -28,6 +28,7 @@ import java.util.Properties; +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.junit.jupiter.api.Assertions.fail; @@ -106,6 +107,7 @@ public void testDelete() { Option outputPayload = payload.combineAndGetUpdateValue(oldRecord, avroSchema, properties); // expect nothing to be committed to table assertFalse(outputPayload.isPresent()); + assertTrue(payload.isDeleted(avroSchema, properties)); } catch (Exception e) { fail("Unexpected exception"); } @@ -142,19 +144,13 @@ public void testPreCombineWithDelete() { deleteRecord.put("Op", "D"); GenericRecord oldRecord = new GenericData.Record(avroSchema); - oldRecord.put("field1", 4); + oldRecord.put("field1", 3); oldRecord.put("Op", "I"); AWSDmsAvroPayload payload = new AWSDmsAvroPayload(Option.of(deleteRecord)); AWSDmsAvroPayload insertPayload = new AWSDmsAvroPayload(Option.of(oldRecord)); - try { - OverwriteWithLatestAvroPayload output = payload.preCombine(insertPayload); - Option outputPayload = output.getInsertValue(avroSchema, properties); - // expect nothing to be committed to table - assertFalse(outputPayload.isPresent()); - } catch (Exception e) { - fail("Unexpected exception"); - } + OverwriteWithLatestAvroPayload output = payload.preCombine(insertPayload); + assertEquals(payload, output); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestPartialUpdateAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestPartialUpdateAvroPayload.java index 2d5bb511981ee..28313f150c81f 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestPartialUpdateAvroPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestPartialUpdateAvroPayload.java @@ -18,11 +18,13 @@ package org.apache.hudi.common.model; +import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.util.Option; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -34,6 +36,8 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + /** * Unit tests {@link TestPartialUpdateAvroPayload}. @@ -147,24 +151,38 @@ public void testDeletedRecord() throws IOException { GenericRecord record2 = new GenericData.Record(schema); record2.put("id", "1"); record2.put("partition", "partition0"); - record2.put("ts", 0L); - record2.put("_hoodie_is_deleted", true); + record2.put("ts", 2L); + record2.put("_hoodie_is_deleted", false); record2.put("city", "NY0"); record2.put("child", Collections.emptyList()); PartialUpdateAvroPayload payload1 = new PartialUpdateAvroPayload(record1, 0L); - PartialUpdateAvroPayload payload2 = new PartialUpdateAvroPayload(delRecord1, 1L); + PartialUpdateAvroPayload delPayload = new PartialUpdateAvroPayload(delRecord1, 1L); + PartialUpdateAvroPayload payload2 = new PartialUpdateAvroPayload(record2, 2L); + + PartialUpdateAvroPayload mergedPayload = payload1.preCombine(delPayload, schema, new Properties()); + assertTrue(HoodieAvroUtils.bytesToAvro(mergedPayload.recordBytes, schema).get("_hoodie_is_deleted").equals(true)); + assertArrayEquals(mergedPayload.recordBytes, delPayload.recordBytes); + + mergedPayload = delPayload.preCombine(payload1, schema, new Properties()); + assertTrue(HoodieAvroUtils.bytesToAvro(mergedPayload.recordBytes, schema).get("_hoodie_is_deleted").equals(true)); + assertArrayEquals(mergedPayload.recordBytes, delPayload.recordBytes); - assertArrayEquals(payload1.preCombine(payload2).recordBytes, payload2.recordBytes); - assertArrayEquals(payload2.preCombine(payload1).recordBytes, payload2.recordBytes); + mergedPayload = payload2.preCombine(delPayload, schema, new Properties()); + assertTrue(HoodieAvroUtils.bytesToAvro(mergedPayload.recordBytes, schema).get("_hoodie_is_deleted").equals(false)); + assertArrayEquals(mergedPayload.recordBytes, payload2.recordBytes); + + mergedPayload = delPayload.preCombine(payload2, schema, new Properties()); + assertTrue(HoodieAvroUtils.bytesToAvro(mergedPayload.recordBytes, schema).get("_hoodie_is_deleted").equals(false)); + assertArrayEquals(mergedPayload.recordBytes, payload2.recordBytes); assertEquals(record1, payload1.getInsertValue(schema).get()); - assertFalse(payload2.getInsertValue(schema).isPresent()); + assertFalse(delPayload.getInsertValue(schema).isPresent()); Properties properties = new Properties(); properties.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, "ts"); assertEquals(payload1.combineAndGetUpdateValue(delRecord1, schema, properties), Option.empty()); - assertFalse(payload2.combineAndGetUpdateValue(record1, schema, properties).isPresent()); + assertFalse(delPayload.combineAndGetUpdateValue(record1, schema, properties).isPresent()); } @Test @@ -205,4 +223,87 @@ public void testUseLatestRecordMetaValue() throws IOException { assertEquals(mergedRecord2.get("_hoodie_commit_time").toString(), record2.get("_hoodie_commit_time").toString()); assertEquals(mergedRecord2.get("_hoodie_commit_seqno").toString(), record2.get("_hoodie_commit_seqno").toString()); } + + /** + * This test is to highlight the gotcha, where there are differences in result of the two queries on the same input data below: + *

+   *   Query A (No precombine):
+   *
+   *   INSERT INTO t1 VALUES (1, 'partition1', 1, false, NY0, ['A']);
+   *   INSERT INTO t1 VALUES (1, 'partition1', 0, false, NY1, ['A']);
+   *   INSERT INTO t1 VALUES (1, 'partition1', 2, false, NULL, ['A']);
+   *
+   *   Final output of Query A:
+   *   (1, 'partition1', 2, false, NY0, ['A'])
+   *
+   *   Query B (preCombine invoked)
+   *   INSERT INTO t1 VALUES (1, 'partition1', 1, false, NULL, ['A']);
+   *   INSERT INTO t1 VALUES (1, 'partition1', 0, false, NY1, ['A']), (1, 'partition1', 2, false, NULL, ['A']);
+   *
+   *   Final output of Query B:
+   *   (1, 'partition1', 2, false, NY1, ['A'])
+   * 
+ * + * + * @throws IOException + */ + @Test + public void testPartialUpdateGotchas() throws IOException { + Properties properties = new Properties(); + properties.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, "ts"); + + GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", "1"); + record1.put("partition", "partition1"); + record1.put("ts", 1L); + record1.put("_hoodie_is_deleted", false); + record1.put("city", "NY0"); + record1.put("child", Arrays.asList("A")); + + GenericRecord record2 = new GenericData.Record(schema); + record2.put("id", "1"); + record2.put("partition", "partition1"); + record2.put("ts", 0L); + record2.put("_hoodie_is_deleted", false); + record2.put("city", "NY1"); + record2.put("child", Arrays.asList("B")); + + GenericRecord record3 = new GenericData.Record(schema); + record3.put("id", "1"); + record3.put("partition", "partition1"); + record3.put("ts", 2L); + record3.put("_hoodie_is_deleted", false); + record3.put("city", null); + record3.put("child", Arrays.asList("A")); + + // define expected outputs + GenericRecord pureCombineOutput = new GenericData.Record(schema); + pureCombineOutput.put("id", "1"); + pureCombineOutput.put("partition", "partition1"); + pureCombineOutput.put("ts", 2L); + pureCombineOutput.put("_hoodie_is_deleted", false); + pureCombineOutput.put("city", "NY0"); + pureCombineOutput.put("child", Arrays.asList("A")); + + GenericRecord outputWithPreCombineUsed = new GenericData.Record(schema); + outputWithPreCombineUsed.put("id", "1"); + outputWithPreCombineUsed.put("partition", "partition1"); + outputWithPreCombineUsed.put("ts", 2L); + outputWithPreCombineUsed.put("_hoodie_is_deleted", false); + outputWithPreCombineUsed.put("city", "NY1"); + outputWithPreCombineUsed.put("child", Arrays.asList("A")); + + PartialUpdateAvroPayload payload2 = new PartialUpdateAvroPayload(record2, 0L); + PartialUpdateAvroPayload payload3 = new PartialUpdateAvroPayload(record3, 2L); + + // query A (no preCombine) + IndexedRecord firstCombineOutput = payload2.combineAndGetUpdateValue(record1, schema, properties).get(); + IndexedRecord secondCombineOutput = payload3.combineAndGetUpdateValue(firstCombineOutput, schema, properties).get(); + assertEquals(pureCombineOutput, secondCombineOutput); + + // query B (preCombine invoked) + PartialUpdateAvroPayload payloadAfterPreCombine = payload3.preCombine(payload2, schema, properties); + IndexedRecord finalOutputWithPreCombine = payloadAfterPreCombine.combineAndGetUpdateValue(record1, schema, properties).get(); + assertEquals(outputWithPreCombineUsed, finalOutputWithPreCombine); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java index 55badd4b4a3d2..f5c3563f06426 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/debezium/TestMySqlDebeziumAvroPayload.java @@ -33,10 +33,12 @@ import java.io.IOException; import java.util.Arrays; import java.util.Objects; +import java.util.Properties; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests {@link MySqlDebeziumAvroPayload}. @@ -100,6 +102,7 @@ public void testMergeWithUpdate() throws IOException { public void testMergeWithDelete() throws IOException { GenericRecord deleteRecord = createRecord(2, Operation.DELETE, "00002.11"); MySqlDebeziumAvroPayload payload = new MySqlDebeziumAvroPayload(deleteRecord, "00002.11"); + assertTrue(payload.isDeleted(avroSchema, new Properties())); GenericRecord existingRecord = createRecord(2, Operation.UPDATE, "00001.111"); Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); 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 7cb8dea9fdbd4..54eca3c6d05d9 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; @@ -37,11 +41,13 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Objects; +import java.util.Properties; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests {@link PostgresDebeziumAvroPayload}. @@ -104,6 +110,7 @@ public void testMergeWithUpdate() throws IOException { public void testMergeWithDelete() throws IOException { GenericRecord deleteRecord = createRecord(2, Operation.DELETE, 100L); PostgresDebeziumAvroPayload payload = new PostgresDebeziumAvroPayload(deleteRecord, 100L); + assertTrue(payload.isDeleted(avroSchema, new Properties())); GenericRecord existingRecord = createRecord(2, Operation.UPDATE, 99L); Option mergedRecord = payload.combineAndGetUpdateValue(existingRecord, avroSchema); @@ -116,6 +123,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(), avroSchema, 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/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 9a2dba04681a5..cacb60b6175ab 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 @@ -101,7 +101,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 { @@ -356,9 +361,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 c6ab11a310fe5..28fb77c57ba89 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 @@ -73,6 +73,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; @@ -607,8 +608,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 5235183d10fb2..87b31531786e7 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.util; import org.apache.hudi.avro.model.HoodieActionInstant; +import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; @@ -48,7 +49,7 @@ import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests for {@link ClusteringUtils}. @@ -128,7 +129,7 @@ public void testGetOldestInstantToRetainForClustering() throws IOException { 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()); + metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant1, Option.empty()); List fileIds2 = new ArrayList<>(); fileIds2.add(UUID.randomUUID().toString()); fileIds2.add(UUID.randomUUID().toString()); @@ -146,29 +147,12 @@ public void testGetOldestInstantToRetainForClustering() throws IOException { 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. + assertTrue(actual.isPresent()); + assertEquals(clusterTime1, actual.get().getTimestamp(), "no clean in timeline, retain first replace commit"); + 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()) @@ -177,11 +161,54 @@ public void testGetOldestInstantToRetainForClustering() throws IOException { .setFilesToBeDeletedPerPartition(new HashMap<>()) .setVersion(CleanPlanV2MigrationHandler.VERSION) .build(); - metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant5, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan2)); - metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant5, Option.empty()); + metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant4, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan1)); + HoodieInstant inflightInstant4 = metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant4, Option.empty()); + HoodieCleanMetadata cleanMetadata = new HoodieCleanMetadata(cleanTime1, 1L, 1, + completedInstant3.getTimestamp(), "", Collections.emptyMap(), 0, Collections.emptyMap()); + metaClient.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant4, + TimelineMetadataUtils.serializeCleanMetadata(cleanMetadata)); metaClient.reloadActiveTimeline(); actual = ClusteringUtils.getOldestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient); - assertEquals(clusterTime3, actual.get().getTimestamp()); + assertEquals(clusterTime3, actual.get().getTimestamp(), + "retain the first replace commit after the earliestInstantToRetain "); + } + + /** test getOldestInstantToRetainForClustering with KEEP_LATEST_FILE_VERSIONS as clean policy */ + @Test + public void testGetOldestInstantToRetainForClusteringKeepFileVersion() 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()); + metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant1, Option.empty()); + + String cleanTime1 = "2"; + HoodieInstant requestedInstant2 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); + HoodieCleanerPlan cleanerPlan1 = new HoodieCleanerPlan(null, clusterTime1, + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name(), Collections.emptyMap(), + CleanPlanV2MigrationHandler.VERSION, Collections.emptyMap(), Collections.emptyList()); + metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant2, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan1)); + HoodieInstant inflightInstant2 = metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant2, Option.empty()); + HoodieCleanMetadata cleanMetadata = new HoodieCleanMetadata(cleanTime1, 1L, 1, + "", "", Collections.emptyMap(), 0, Collections.emptyMap()); + metaClient.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant2, + TimelineMetadataUtils.serializeCleanMetadata(cleanMetadata)); + metaClient.reloadActiveTimeline(); + + List fileIds2 = new ArrayList<>(); + fileIds2.add(UUID.randomUUID().toString()); + fileIds2.add(UUID.randomUUID().toString()); + String clusterTime2 = "3"; + HoodieInstant requestedInstant3 = createRequestedReplaceInstant(partitionPath1, clusterTime2, fileIds2); + HoodieInstant inflightInstant3 = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant3, Option.empty()); + metaClient.getActiveTimeline().transitionReplaceInflightToComplete(inflightInstant3, Option.empty()); + metaClient.reloadActiveTimeline(); + + Option actual = ClusteringUtils.getOldestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient); + assertEquals(clusterTime2, actual.get().getTimestamp(), + "retain the first replace commit after the last complete clean "); } private void validateClusteringInstant(List fileIds, String partitionPath, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestIterators.java b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestIterators.java new file mode 100644 index 0000000000000..6e9d43388b12f --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/collection/TestIterators.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util.collection; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.util.CollectionUtils.toStream; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestIterators { + + @Test + public void testFlatteningIterator() { + List> listOfList = + Arrays.asList( + Arrays.asList(0), + Arrays.asList(1, 2), + Collections.emptyList(), + Arrays.asList(3, 4, 5) + ); + + List flattenedList = + toStream(new FlatteningIterator<>(new MappingIterator<>(listOfList.iterator(), List::iterator))) + .collect(Collectors.toList()); + + assertEquals(Arrays.asList(0, 1, 2, 3, 4, 5), flattenedList); + } +} 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-common/pom.xml b/hudi-examples/hudi-examples-common/pom.xml index dac3d391e8563..d9a82506f7717 100644 --- a/hudi-examples/hudi-examples-common/pom.xml +++ b/hudi-examples/hudi-examples-common/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-examples/hudi-examples-flink/pom.xml b/hudi-examples/hudi-examples-flink/pom.xml index 306c9a0076d38..928a986c7689a 100644 --- a/hudi-examples/hudi-examples-flink/pom.xml +++ b/hudi-examples/hudi-examples-flink/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-examples/hudi-examples-java/pom.xml b/hudi-examples/hudi-examples-java/pom.xml index 48dcf1d9d5cbb..22388e8d24c63 100644 --- a/hudi-examples/hudi-examples-java/pom.xml +++ b/hudi-examples/hudi-examples-java/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-examples/hudi-examples-spark/pom.xml b/hudi-examples/hudi-examples-spark/pom.xml index 50fda6f7e5723..04814c622f218 100644 --- a/hudi-examples/hudi-examples-spark/pom.xml +++ b/hudi-examples/hudi-examples-spark/pom.xml @@ -21,7 +21,7 @@ hudi-examples org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java index 325aad437ade7..3eea50e4961f4 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/quickstart/HoodieSparkQuickstart.java @@ -115,7 +115,7 @@ public static Dataset insertData(SparkSession spark, JavaSparkContext jsc, List inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20)); Dataset df = spark.read().json(jsc.parallelize(inserts, 1)); - df.write().format("org.apache.hudi") + df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") @@ -135,7 +135,7 @@ public static Dataset insertOverwriteData(SparkSession spark, JavaSparkCont List inserts = dataGen.convertToStringList(dataGen.generateInsertsOnPartition(commitTime, 20, HoodieExampleDataGenerator.DEFAULT_THIRD_PARTITION_PATH)); Dataset df = spark.read().json(jsc.parallelize(inserts, 1)); - df.write().format("org.apache.hudi") + df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option("hoodie.datasource.write.operation", WriteOperationType.INSERT_OVERWRITE.name()) .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") @@ -154,7 +154,7 @@ public static void queryData(SparkSession spark, JavaSparkContext jsc, String ta HoodieExampleDataGenerator dataGen) { Dataset roViewDF = spark .read() - .format("org.apache.hudi") + .format("hudi") .load(tablePath + "/*/*/*/*"); roViewDF.createOrReplaceTempView("hudi_ro_table"); @@ -186,7 +186,7 @@ public static Dataset updateData(SparkSession spark, JavaSparkContext jsc, String commitTime = Long.toString(System.currentTimeMillis()); List updates = dataGen.convertToStringList(dataGen.generateUniqueUpdates(commitTime)); Dataset df = spark.read().json(jsc.parallelize(updates, 1)); - df.write().format("org.apache.hudi") + df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") @@ -202,12 +202,12 @@ public static Dataset updateData(SparkSession spark, JavaSparkContext jsc, */ public static Dataset delete(SparkSession spark, String tablePath, String tableName) { - Dataset roViewDF = spark.read().format("org.apache.hudi").load(tablePath + "/*/*/*/*"); + Dataset roViewDF = spark.read().format("hudi").load(tablePath + "/*/*/*/*"); roViewDF.createOrReplaceTempView("hudi_ro_table"); Dataset toBeDeletedDf = spark.sql("SELECT begin_lat, begin_lon, driver, end_lat, end_lon, fare, partitionpath, rider, ts, uuid FROM hudi_ro_table limit 2"); Dataset df = toBeDeletedDf.select("uuid", "partitionpath", "ts"); - df.write().format("org.apache.hudi") + df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") @@ -224,7 +224,7 @@ public static Dataset delete(SparkSession spark, String tablePath, String t */ public static void deleteByPartition(SparkSession spark, String tablePath, String tableName) { Dataset df = spark.emptyDataFrame(); - df.write().format("org.apache.hudi") + df.write().format("hudi") .options(QuickstartUtils.getQuickstartWriteConfigs()) .option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts") .option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid") @@ -253,7 +253,7 @@ public static void incrementalQuery(SparkSession spark, String tablePath, String // incrementally query data Dataset incViewDF = spark .read() - .format("org.apache.hudi") + .format("hudi") .option("hoodie.datasource.query.type", "incremental") .option("hoodie.datasource.read.begin.instanttime", beginTime) .load(tablePath); @@ -278,7 +278,7 @@ public static void pointInTimeQuery(SparkSession spark, String tablePath, String String endTime = commits.get(commits.size() - 1); // commit time we are interested in //incrementally query data - Dataset incViewDF = spark.read().format("org.apache.hudi") + Dataset incViewDF = spark.read().format("hudi") .option("hoodie.datasource.query.type", "incremental") .option("hoodie.datasource.read.begin.instanttime", beginTime) .option("hoodie.datasource.read.end.instanttime", endTime) diff --git a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala index 33c085cba3eb6..fe5164f33a55f 100644 --- a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala +++ b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala @@ -30,16 +30,16 @@ import org.apache.spark.sql.SparkSession import scala.collection.JavaConversions._ /** - * Simple examples of [[org.apache.hudi.DefaultSource]] - * - * To run this example, you should - * 1. For running in IDE, set VM options `-Dspark.master=local[2]` - * 2. For running in shell, using `spark-submit` - * - * Usage: HoodieWriteClientExample . - * and describe root path of hudi and table name - * for example, `HoodieDataSourceExample file:///tmp/hoodie/hudi_cow_table hudi_cow_table` - */ + * Simple examples of [[org.apache.hudi.DefaultSource]] + * + * To run this example, you should + * 1. For running in IDE, set VM options `-Dspark.master=local[2]` + * 2. For running in shell, using `spark-submit` + * + * Usage: HoodieWriteClientExample . + * and describe root path of hudi and table name + * for example, `HoodieDataSourceExample file:///tmp/hoodie/hudi_cow_table hudi_cow_table` + */ object HoodieDataSourceExample { def main(args: Array[String]): Unit = { @@ -75,7 +75,7 @@ object HoodieDataSourceExample { val commitTime: String = System.currentTimeMillis().toString val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20)) val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1)) - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). @@ -90,9 +90,9 @@ object HoodieDataSourceExample { */ def queryData(spark: SparkSession, tablePath: String, tableName: String, dataGen: HoodieExampleDataGenerator[HoodieAvroPayload]): Unit = { val roViewDF = spark. - read. - format("org.apache.hudi"). - load(tablePath + "/*/*/*/*") + read. + format("hudi"). + load(tablePath + "/*/*/*/*") roViewDF.createOrReplaceTempView("hudi_ro_table") @@ -120,7 +120,7 @@ object HoodieDataSourceExample { val commitTime: String = System.currentTimeMillis().toString val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10)) val df = spark.read.json(spark.sparkContext.parallelize(updates, 1)) - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). @@ -135,11 +135,11 @@ object HoodieDataSourceExample { */ def delete(spark: SparkSession, tablePath: String, tableName: String): Unit = { - val roViewDF = spark.read.format("org.apache.hudi").load(tablePath + "/*/*/*/*") + val roViewDF = spark.read.format("hudi").load(tablePath + "/*/*/*/*") roViewDF.createOrReplaceTempView("hudi_ro_table") val df = spark.sql("select uuid, partitionpath, ts from hudi_ro_table limit 2") - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). @@ -155,7 +155,7 @@ object HoodieDataSourceExample { */ def deleteByPartition(spark: SparkSession, tablePath: String, tableName: String): Unit = { val df = spark.emptyDataFrame - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). @@ -179,10 +179,10 @@ object HoodieDataSourceExample { // incrementally query data val incViewDF = spark. - read. - format("org.apache.hudi"). - option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL). - option(BEGIN_INSTANTTIME.key, beginTime). + read. + format("hudi"). + option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL). + option(BEGIN_INSTANTTIME.key, beginTime). load(tablePath) incViewDF.createOrReplaceTempView("hudi_incr_table") spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0").show() @@ -200,11 +200,11 @@ object HoodieDataSourceExample { val endTime = commits(commits.length - 2) // commit time we are interested in //incrementally query data - val incViewDF = spark.read.format("org.apache.hudi"). - option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL). - option(BEGIN_INSTANTTIME.key, beginTime). - option(END_INSTANTTIME.key, endTime). - load(tablePath) + val incViewDF = spark.read.format("hudi"). + option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL). + option(BEGIN_INSTANTTIME.key, beginTime). + option(END_INSTANTTIME.key, endTime). + load(tablePath) incViewDF.createOrReplaceTempView("hudi_incr_table") spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0").show() } diff --git a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala index 8a2c8715b30eb..4802632ad035a 100644 --- a/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala +++ b/hudi-examples/hudi-examples-spark/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala @@ -85,7 +85,7 @@ object HoodieMorCompactionJob { val commitTime: String = System.currentTimeMillis().toString val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20)) val df = spark.read.json(spark.sparkContext.parallelize(inserts.asScala, 1)) - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). @@ -101,7 +101,7 @@ object HoodieMorCompactionJob { val commitTime: String = System.currentTimeMillis().toString val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10)) val df = spark.read.json(spark.sparkContext.parallelize(updates.asScala, 1)) - df.write.format("org.apache.hudi"). + df.write.format("hudi"). options(getQuickstartWriteConfigs). option(PRECOMBINE_FIELD.key, "ts"). option(RECORDKEY_FIELD.key, "uuid"). 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 96c73dd240bc3..8089f86928034 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 @@ -20,6 +20,7 @@ import org.apache.hudi.client.SparkRDDReadClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness; import org.apache.hudi.testutils.providers.SparkProvider; import org.apache.spark.HoodieSparkKryoRegistrar$; @@ -56,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-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py b/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py index c1303f6365d20..caad76ca6009b 100644 --- a/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py +++ b/hudi-examples/hudi-examples-spark/src/test/python/HoodiePySparkQuickstart.py @@ -256,7 +256,6 @@ def insertOverwrite(self): .appName("Hudi Spark basic example") \ .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") \ .config("spark.kryo.registrator", "org.apache.spark.HoodieSparkKryoRegistrar") \ - .config("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension") \ .config("spark.kryoserializer.buffer.max", "512m") \ .config("spark.sql.extensions", "org.apache.spark.sql.hudi.HoodieSparkSessionExtension") \ .getOrCreate() diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml index dc5e3390583f1..afc353f01b273 100644 --- a/hudi-examples/pom.xml +++ b/hudi-examples/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml index 3701998bbaa6e..0c18446ab6f47 100644 --- a/hudi-flink-datasource/hudi-flink/pom.xml +++ b/hudi-flink-datasource/hudi-flink/pom.xml @@ -22,12 +22,12 @@ hudi-flink-datasource org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink - 0.13.0-SNAPSHOT + 0.13.1 jar 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 c1812b79e3819..f3ce6c33cc9a6 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 @@ -25,11 +25,13 @@ import org.apache.hudi.common.model.EventTimeAvroPayload; import org.apache.hudi.common.model.HoodieAvroRecordMerger; import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.HoodieSyncTableStrategy; import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieSyncTableStrategy; 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.HoodieClusteringConfig; +import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; @@ -49,11 +51,8 @@ import java.util.Map; import java.util.Set; +import static org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode.data_before_after; import static org.apache.hudi.common.util.PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH; -import static org.apache.hudi.config.HoodieClusteringConfig.DAYBASED_LOOKBACK_PARTITIONS; -import static org.apache.hudi.config.HoodieClusteringConfig.PARTITION_FILTER_BEGIN_PARTITION; -import static org.apache.hudi.config.HoodieClusteringConfig.PARTITION_FILTER_END_PARTITION; -import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST; /** * Hoodie Flink config options. @@ -168,12 +167,11 @@ private FlinkOptions() { public static final ConfigOption SUPPLEMENTAL_LOGGING_MODE = ConfigOptions .key("cdc.supplemental.logging.mode") .stringType() - .defaultValue("cdc_data_before_after") // default record all the change log images + .defaultValue(data_before_after.name()) .withFallbackKeys(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE.key()) - .withDescription("The supplemental logging mode:" - + "1) 'cdc_op_key': persist the 'op' and the record key only," - + "2) 'cdc_data_before': persist the additional 'before' image," - + "3) 'cdc_data_before_after': persist the 'before' and 'after' images at the same time"); + .withDescription("Setting 'op_key_only' persists the 'op' and the record key only, " + + "setting 'data_before' persists the additional 'before' image, " + + "and setting 'data_before_after' persists the additional 'before' and 'after' images."); // ------------------------------------------------------------------------ // Metadata table Options @@ -297,10 +295,10 @@ private FlinkOptions() { .key("read.streaming.skip_compaction") .booleanType() .defaultValue(false)// default read as batch - .withDescription("Whether to skip compaction instants for streaming read,\n" - + "there are two cases that this option can be used to avoid reading duplicates:\n" - + "1) you are definitely sure that the consumer reads faster than any compaction instants, " - + "usually with delta time compaction strategy that is long enough, for e.g, one week;\n" + .withDescription("Whether to skip compaction instants and avoid reading compacted base files for streaming read to improve read performance.\n" + + "There are two cases that this option can be used to avoid reading duplicates:\n" + + "1) you are definitely sure that the consumer reads [faster than/completes before] any compaction instants " + + "when " + HoodieCompactionConfig.PRESERVE_COMMIT_METADATA.key() + " is set to false.\n" + "2) changelog mode is enabled, this option is a solution to keep data integrity"); // this option is experimental @@ -308,8 +306,11 @@ private FlinkOptions() { .key("read.streaming.skip_clustering") .booleanType() .defaultValue(false) - .withDescription("Whether to skip clustering instants for streaming read,\n" - + "to avoid reading duplicates"); + .withDescription("Whether to skip clustering instants to avoid reading base files of clustering operations for streaming read " + + "to improve read performance.\n" + + "This option toggled to true to avoid duplicates when: \n" + + "1) you are definitely sure that the consumer reads [faster than/completes before] any clustering instants " + + "when " + HoodieClusteringConfig.PRESERVE_COMMIT_METADATA.key() + " is set to false.\n"); public static final String START_COMMIT_EARLIEST = "earliest"; public static final ConfigOption READ_START_COMMIT = ConfigOptions @@ -381,9 +382,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()) @@ -438,7 +439,9 @@ private FlinkOptions() { .key(HoodieWriteConfig.KEYGENERATOR_TYPE.key()) .stringType() .defaultValue(KeyGeneratorType.SIMPLE.name()) - .withDescription("Key generator type, that implements will extract the key out of incoming record"); + .withDescription("Key generator type, that implements will extract the key out of incoming record. " + + "**Note** This is being actively worked on. Please use " + + "`hoodie.datasource.write.keygenerator.class` instead."); public static final String PARTITION_FORMAT_HOUR = "yyyyMMddHH"; public static final String PARTITION_FORMAT_DAY = "yyyyMMdd"; @@ -699,6 +702,36 @@ private FlinkOptions() { .defaultValue(2) .withDescription("Number of partitions to list to create ClusteringPlan, default is 2"); + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigOptions + .key("clustering.plan.strategy.daybased.skipfromlatest.partitions") + .intType() + .defaultValue(0) + .withDescription("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_CLUSTER_BEGIN_PARTITION = ConfigOptions + .key("clustering.plan.strategy.cluster.begin.partition") + .stringType() + .defaultValue("") + .withDescription("Begin partition used to filter partition (inclusive)"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_CLUSTER_END_PARTITION = ConfigOptions + .key("clustering.plan.strategy.cluster.end.partition") + .stringType() + .defaultValue("") + .withDescription("End partition used to filter partition (inclusive)"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_PARTITION_REGEX_PATTERN = ConfigOptions + .key("clustering.plan.strategy.partition.regex.pattern") + .stringType() + .defaultValue("") + .withDescription("Filter clustering partitions that matched regex pattern"); + + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_PARTITION_SELECTED = ConfigOptions + .key("clustering.plan.strategy.partition.selected") + .stringType() + .defaultValue("") + .withDescription("Partitions to run clustering"); + public static final ConfigOption CLUSTERING_PLAN_STRATEGY_CLASS = ConfigOptions .key("clustering.plan.strategy.class") .stringType() @@ -713,10 +746,10 @@ private FlinkOptions() { .defaultValue(ClusteringPlanPartitionFilterMode.NONE.name()) .withDescription("Partition filter mode used in the creation of clustering plan. Available values are - " + "NONE: do not filter table partition and thus the clustering plan will include all partitions that have clustering candidate." - + "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() + "']." + + "RECENT_DAYS: keep a continuous range of partitions, worked together with configs '" + CLUSTERING_TARGET_PARTITIONS.key() + "' and '" + + CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST.key() + "." + + "SELECTED_PARTITIONS: keep partitions that are in the specified range ['" + CLUSTERING_PLAN_STRATEGY_CLUSTER_BEGIN_PARTITION.key() + "', '" + + CLUSTERING_PLAN_STRATEGY_CLUSTER_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."); @@ -732,12 +765,6 @@ private FlinkOptions() { .defaultValue(600L) // default 600 MB .withDescription("Files smaller than the size specified here are candidates for clustering, default 600 MB"); - public static final ConfigOption CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigOptions - .key("clustering.plan.strategy.daybased.skipfromlatest.partitions") - .intType() - .defaultValue(0) - .withDescription("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan"); - public static final ConfigOption CLUSTERING_SORT_COLUMNS = ConfigOptions .key("clustering.plan.strategy.sort.columns") .stringType() 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 af3e25ef2c0bc..8969d055dee5f 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 @@ -23,8 +23,10 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; 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.ConfigOption; @@ -53,8 +55,7 @@ public static boolean insertClustering(Configuration conf) { public static boolean isAppendMode(Configuration conf) { // 1. inline clustering is supported for COW table; // 2. async clustering is supported for both COW and MOR table - return isCowTable(conf) && isInsertOperation(conf) && !conf.getBoolean(FlinkOptions.INSERT_CLUSTER) - || needsScheduleClustering(conf); + return isInsertOperation(conf) && ((isCowTable(conf) && !conf.getBoolean(FlinkOptions.INSERT_CLUSTER)) || isMorTable(conf)); } /** @@ -210,7 +211,7 @@ public static boolean hasNoSpecificReadCommits(Configuration conf) { */ public static HoodieCDCSupplementalLoggingMode getCDCSupplementalLoggingMode(Configuration conf) { String mode = conf.getString(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE); - return HoodieCDCSupplementalLoggingMode.parse(mode); + return HoodieCDCSupplementalLoggingMode.valueOf(mode); } /** @@ -227,6 +228,21 @@ public static boolean isIncrementalQuery(Configuration conf) { return conf.getOptional(FlinkOptions.READ_START_COMMIT).isPresent() || conf.getOptional(FlinkOptions.READ_END_COMMIT).isPresent(); } + /** + * 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())); + } + + /** + * Returns whether to commit even when current batch has no data, for flink defaults false + */ + public static boolean allowCommitOnEmptyBatch(Configuration conf) { + return conf.getBoolean(HoodieWriteConfig.ALLOW_EMPTY_COMMIT.key(), false); + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java index 638fe9fdab286..b674df1771504 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java @@ -51,7 +51,7 @@ public class CleanFunction extends AbstractRichFunction private NonThrownExecutor executor; - private volatile boolean isCleaning; + protected volatile boolean isCleaning; public CleanFunction(Configuration conf) { this.conf = conf; @@ -64,7 +64,14 @@ public void open(Configuration parameters) throws Exception { this.executor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build(); String instantTime = HoodieActiveTimeline.createNewInstantTime(); LOG.info(String.format("exec clean with instant time %s...", instantTime)); - executor.execute(() -> writeClient.clean(instantTime), "wait for cleaning finish"); + executor.execute(() -> { + this.isCleaning = true; + try { + this.writeClient.clean(instantTime); + } finally { + this.isCleaning = false; + } + }, "wait for cleaning finish"); } @Override 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 7f0285e83ed2a..c37b2325ca76a 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 @@ -31,7 +31,6 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.sink.common.AbstractStreamWriteFunction; import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.table.action.commit.FlinkWriteHelper; @@ -53,7 +52,6 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Properties; import java.util.Random; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -431,10 +429,8 @@ private boolean flushBucket(DataBucket bucket) { List records = bucket.writeBuffer(); ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records"); if (config.getBoolean(FlinkOptions.PRE_COMBINE)) { - Properties props = new Properties(); - config.addAllToProperties(props); records = (List) FlinkWriteHelper.newInstance() - .deduplicateRecords(records, (HoodieIndex) null, -1, this.writeClient.getConfig().getSchema(), props, recordMerger); + .deduplicateRecords(records, null, -1, this.writeClient.getConfig().getSchema(), this.writeClient.getConfig().getProps(), recordMerger); } bucket.preWrite(records); final List writeStatus = new ArrayList<>(writeFunction.apply(records, instant)); @@ -469,10 +465,8 @@ private void flushRemaining(boolean endInput) { List records = bucket.writeBuffer(); if (records.size() > 0) { if (config.getBoolean(FlinkOptions.PRE_COMBINE)) { - Properties props = new Properties(); - config.addAllToProperties(props); records = (List) FlinkWriteHelper.newInstance() - .deduplicateRecords(records, (HoodieIndex) null, -1, this.writeClient.getConfig().getSchema(), props, recordMerger); + .deduplicateRecords(records, null, -1, this.writeClient.getConfig().getSchema(), this.writeClient.getConfig().getProps(), recordMerger); } bucket.preWrite(records); writeStatus.addAll(writeFunction.apply(records, currentInstant)); 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 738a0089ef9c7..73cb3c22b1341 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 @@ -33,6 +33,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; @@ -335,7 +336,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) { @@ -501,7 +504,7 @@ private boolean commitInstant(String instant, long checkpointId) { .flatMap(Collection::stream) .collect(Collectors.toList()); - if (writeResults.size() == 0) { + if (writeResults.size() == 0 && !OptionsResolver.allowCommitOnEmptyBatch(conf)) { // No data has written, reset the buffer and returns early reset(); // Send commit ack event to the write function to unblock the flushing 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 c989b4eb29ae2..afd55639c3c15 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 @@ -135,8 +135,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; } /** @@ -156,17 +157,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/RowDataKeyGen.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/RowDataKeyGen.java index a2414abc3de21..00b2c49e98045 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; @@ -56,6 +57,8 @@ public class RowDataKeyGen implements Serializable { private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/"; + private final boolean hasRecordKey; + private final String[] recordKeyFields; private final String[] partitionPathFields; @@ -64,6 +67,7 @@ public class RowDataKeyGen implements Serializable { private final boolean hiveStylePartitioning; private final boolean encodePartitionPath; + private final boolean consistentLogicalTimestampEnabled; private final Option keyGenOpt; @@ -82,6 +86,7 @@ private RowDataKeyGen( RowType rowType, boolean hiveStylePartitioning, boolean encodePartitionPath, + boolean consistentLogicalTimestampEnabled, Option keyGenOpt) { this.recordKeyFields = recordKeys.split(","); this.partitionPathFields = partitionFields.split(","); @@ -90,7 +95,12 @@ private RowDataKeyGen( this.hiveStylePartitioning = hiveStylePartitioning; this.encodePartitionPath = encodePartitionPath; - if (this.recordKeyFields.length == 1) { + this.consistentLogicalTimestampEnabled = consistentLogicalTimestampEnabled; + + this.hasRecordKey = hasRecordKey(fieldNames); + if (!hasRecordKey) { + this.recordKeyProjection = null; + } else if (this.recordKeyFields.length == 1) { // efficient code path this.simpleRecordKey = true; int recordKeyIdx = fieldNames.indexOf(this.recordKeyFields[0]); @@ -115,6 +125,14 @@ private RowDataKeyGen( this.keyGenOpt = keyGenOpt; } + /** + * Checks whether user provides any record key. + */ + private boolean hasRecordKey(List fieldNames) { + return recordKeyFields.length != 1 + || fieldNames.contains(recordKeyFields[0]); + } + public static RowDataKeyGen instance(Configuration conf, RowType rowType) { Option keyGeneratorOpt = Option.empty(); if (TimestampBasedAvroKeyGenerator.class.getName().equals(conf.getString(FlinkOptions.KEYGEN_CLASS_NAME))) { @@ -124,9 +142,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) { @@ -134,11 +153,15 @@ public HoodieKey getHoodieKey(RowData rowData) { } public String getRecordKey(RowData rowData) { - if (this.simpleRecordKey) { - return getRecordKey(recordKeyFieldGetter.getFieldOrNull(rowData), this.recordKeyFields[0]); + if (!hasRecordKey) { + // should be optimized to unique values that can be easily calculated with low cost + // for e.g, fileId + auto inc integer + return EMPTY_RECORDKEY_PLACEHOLDER; + } else if (this.simpleRecordKey) { + 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 +178,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 +203,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 +239,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 eb567d89f18ae..7fde3e87aa419 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()); + ClusteringUtil.rollbackClustering(table, writeClient, instant); + return; + } + HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); writeMetadata.setWriteStatuses(statuses); writeMetadata.setWriteStats(statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList())); @@ -168,7 +179,7 @@ private void doCommit(String instant, HoodieClusteringPlan clusteringPlan, List< TableServiceType.CLUSTER, writeMetadata.getCommitMetadata().get(), table, instant); // whether to clean up the input base parquet files used for clustering - if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) { + if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && !isCleaning) { LOG.info("Running inline clean"); this.writeClient.clean(); } 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 85fd846b24726..96462c2b345b6 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 @@ -18,11 +18,13 @@ package org.apache.hudi.sink.clustering; +import org.apache.hudi.adapter.MaskingOutputAdapter; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.client.FlinkTaskContextSupplier; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.utils.ConcatenatingIterator; +import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.model.ClusteringGroupInfo; import org.apache.hudi.common.model.ClusteringOperation; import org.apache.hudi.common.model.HoodieRecord; @@ -33,7 +35,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; @@ -60,12 +61,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.streaming.api.graph.StreamConfig; 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.streaming.runtime.tasks.StreamTask; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.binary.BinaryRowData; import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; @@ -146,9 +147,13 @@ public ClusteringOperator(Configuration conf, RowType rowType) { // target size should larger than small file limit this.conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT.key(), - this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES) > this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT) - ? this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT) - : this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES)); + Math.min(this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES) / 1024 / 1024, + this.conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT))); + } + + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); } @Override @@ -174,16 +179,6 @@ public void open() throws Exception { this.collector = new StreamRecordCollector<>(output); } - @Override - 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(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java index 739cb29052a31..1b250bce9a3e4 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java @@ -60,9 +60,6 @@ public class FlinkClusteringConfig extends Configuration { @Parameter(names = {"--clustering-tasks"}, description = "Parallelism of tasks that do actual clustering, default is -1") public Integer clusteringTasks = -1; - @Parameter(names = {"--compaction-max-memory"}, description = "Max memory in MB for compaction spillable map, default 100MB.") - public Integer compactionMaxMemory = 100; - @Parameter(names = {"--clean-retain-commits"}, description = "Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled).\n" + "This also directly translates into how much you can incrementally pull on this table, default 10") @@ -104,17 +101,32 @@ public class FlinkClusteringConfig extends Configuration { @Parameter(names = {"--sort-columns"}, description = "Columns to sort the data by when clustering.") public String sortColumns = ""; + @Parameter(names = {"--sort-memory"}, description = "Sort memory in MB, default 128MB.") + public Integer sortMemory = 128; + @Parameter(names = {"--max-num-groups"}, description = "Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. default 30") public Integer maxNumGroups = 30; @Parameter(names = {"--target-partitions"}, description = "Number of partitions to list to create ClusteringPlan, default 2") public Integer targetPartitions = 2; + @Parameter(names = {"--cluster-begin-partition"}, description = "Begin partition used to filter partition (inclusive)") + public String clusterBeginPartition = ""; + + @Parameter(names = {"--cluster-end-partition"}, description = "End partition used to filter partition (inclusive)") + public String clusterEndPartition = ""; + + @Parameter(names = {"--partition-regex-pattern"}, description = "Filter clustering partitions that matched regex pattern") + public String partitionRegexPattern = ""; + + @Parameter(names = {"--partition-selected"}, description = "Partitions to run clustering") + public String partitionSelected = ""; + public static final String SEQ_FIFO = "FIFO"; public static final String SEQ_LIFO = "LIFO"; @Parameter(names = {"--seq"}, description = "Clustering plan execution sequence, two options are supported:\n" - + "1). FIFO: execute the oldest plan first;\n" - + "2). LIFO: execute the latest plan first, by default FIFO") + + "1). FIFO: execute the oldest plan first, by default FIFO;\n" + + "2). LIFO: execute the latest plan first") public String clusteringSeq = SEQ_FIFO; @Parameter(names = {"--service"}, description = "Flink Clustering runs in service mode, disable by default") @@ -165,7 +177,6 @@ public static Configuration toFlinkConfig(FlinkClusteringConfig config) { conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, config.archiveMaxCommits); conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, config.archiveMinCommits); conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, config.cleanRetainCommits); - conf.setInteger(FlinkOptions.COMPACTION_MAX_MEMORY, config.compactionMaxMemory); conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, config.clusteringDeltaCommits); conf.setInteger(FlinkOptions.CLUSTERING_TASKS, config.clusteringTasks); conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLASS, config.planStrategyClass); @@ -173,7 +184,12 @@ public static Configuration toFlinkConfig(FlinkClusteringConfig config) { conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, config.skipFromLatestPartitions); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_BEGIN_PARTITION, config.clusterBeginPartition); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_END_PARTITION, config.clusterEndPartition); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_REGEX_PATTERN, config.partitionRegexPattern); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_SELECTED, config.partitionSelected); conf.setString(FlinkOptions.CLUSTERING_SORT_COLUMNS, config.sortColumns); + conf.setInteger(FlinkOptions.WRITE_SORT_MEMORY, config.sortMemory); conf.setInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS, config.maxNumGroups); conf.setInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS, config.targetPartitions); conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnable); 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 fa4c3db86eac1..04cab0b1eff30 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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.StreamWriteOperatorCoordinator; @@ -46,7 +47,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Objects; /** * Base infrastructures for streaming writer function. @@ -194,8 +194,11 @@ public void setOperatorEventGateway(OperatorEventGateway operatorEventGateway) { private void restoreWriteMetadata() throws Exception { boolean eventSent = false; + HoodieTimeline pendingTimeline = this.metaClient.getActiveTimeline().filterPendingExcludingCompaction(); for (WriteMetadataEvent event : this.writeMetadataState.get()) { - if (Objects.equals(this.currentInstant, event.getInstantTime())) { + // Must filter out the completed instants in case it is a partial failover, + // the write status should not be accumulated in such case. + if (pendingTimeline.containsInstant(event.getInstantTime())) { // Reset taskID for event event.setTaskID(taskID); // The checkpoint succeed but the meta does not commit, 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 e910d5863972f..66743264457c4 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 @@ -18,6 +18,7 @@ package org.apache.hudi.sink.compact; +import org.apache.hudi.adapter.MaskingOutputAdapter; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.CompactionOperation; @@ -32,10 +33,11 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.graph.StreamConfig; 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.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.table.runtime.operators.TableStreamOperator; import org.apache.flink.table.runtime.util.StreamRecordCollector; import org.apache.flink.util.Collector; @@ -88,6 +90,11 @@ public CompactOperator(Configuration conf) { this.asyncCompaction = OptionsResolver.needsAsyncCompaction(conf); } + @Override + public void setup(StreamTask containingTask, StreamConfig config, Output> output) { + super.setup(containingTask, config, new MaskingOutputAdapter<>(output)); + } + @Override public void open() throws Exception { this.taskID = getRuntimeContext().getIndexOfThisSubtask(); @@ -98,16 +105,6 @@ public void open() throws Exception { this.collector = new StreamRecordCollector<>(output); } - @Override - 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(); 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 1e05dce6076c9..828aa3c42651f 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()); @@ -164,7 +175,7 @@ private void doCommit(String instant, Collection events) this.writeClient.commitCompaction(instant, metadata, Option.empty()); // Whether to clean up the old log file when compaction - if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) { + if (!conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED) && !isCleaning) { this.writeClient.clean(); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java index 0308d24633349..1019e64f772f6 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/FlinkCompactionConfig.java @@ -96,8 +96,8 @@ public class FlinkCompactionConfig extends Configuration { public static final String SEQ_FIFO = "FIFO"; public static final String SEQ_LIFO = "LIFO"; @Parameter(names = {"--seq"}, description = "Compaction plan execution sequence, two options are supported:\n" - + "1). FIFO: execute the oldest plan first;\n" - + "2). LIFO: execute the latest plan first, by default LIFO") + + "1). FIFO: execute the oldest plan first, by default FIFO;\n" + + "2). LIFO: execute the latest plan first") public String compactionSeq = SEQ_FIFO; @Parameter(names = {"--service"}, description = "Flink Compaction runs in service mode, disable by default") 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 addabcdc4c803..2fe370accb1a1 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 @@ -21,6 +21,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieException; @@ -92,14 +93,13 @@ public void close() { // ------------------------------------------------------------------------- /** - * Initialize the message bus, would keep all the messages. + * Initialize the message bus, would clean all the messages * *

This expects to be called by the driver. */ public void bootstrap() throws IOException { - if (!fs.exists(path)) { - fs.mkdirs(path); - } + fs.delete(path, true); + fs.mkdirs(path); } public void startInstant(String instant) { @@ -201,6 +201,11 @@ public boolean isAborted(String instant) { return this.messages.stream().anyMatch(ckpMsg -> instant.equals(ckpMsg.getInstant()) && ckpMsg.isAborted()); } + @VisibleForTesting + public List getInstantCache() { + return this.instantCache; + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- @@ -221,6 +226,10 @@ private Path fullPath(String fileName) { } private List scanCkpMetadata(Path ckpMetaPath) throws IOException { + // This is required when the storage is minio + if (!this.fs.exists(ckpMetaPath)) { + return new ArrayList<>(); + } return Arrays.stream(this.fs.listStatus(ckpMetaPath)).map(CkpMessage::new) .collect(Collectors.groupingBy(CkpMessage::getInstant)).values().stream() .map(messages -> messages.stream().reduce((x, y) -> { 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/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index 9ee93719813a9..ce4113e760344 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -63,6 +63,7 @@ public class FileIndex { private final Path path; private final RowType rowType; private final HoodieMetadataConfig metadataConfig; + private final org.apache.hadoop.conf.Configuration hadoopConf; private final boolean dataSkippingEnabled; private List partitionPaths; // cache of partition paths private List filters; // push down filters @@ -71,6 +72,7 @@ public class FileIndex { private FileIndex(Path path, Configuration conf, RowType rowType) { this.path = path; this.rowType = rowType; + this.hadoopConf = HadoopConfigurations.getHadoopConf(conf); this.metadataConfig = metadataConfig(conf); this.dataSkippingEnabled = conf.getBoolean(FlinkOptions.READ_DATA_SKIPPING_ENABLED); this.tableExists = StreamerUtil.tableExists(path.toString(), HadoopConfigurations.getHadoopConf(conf)); @@ -137,7 +139,7 @@ public FileStatus[] getFilesInPartitions() { return new FileStatus[0]; } String[] partitions = getOrBuildPartitionPaths().stream().map(p -> fullPartitionPath(path, p)).toArray(String[]::new); - FileStatus[] allFileStatus = FSUtils.getFilesInPartitions(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString(), + FileStatus[] allFileStatus = FSUtils.getFilesInPartitions(new HoodieFlinkEngineContext(hadoopConf), metadataConfig, path.toString(), partitions) .values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new); Set candidateFiles = candidateFilesInMetadataTable(allFileStatus); @@ -285,7 +287,7 @@ public List getOrBuildPartitionPaths() { return this.partitionPaths; } this.partitionPaths = this.tableExists - ? FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString()) + ? FSUtils.getAllPartitionPaths(new HoodieFlinkEngineContext(hadoopConf), metadataConfig, path.toString()) : Collections.emptyList(); return this.partitionPaths; } 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 c3e7fe29d4327..29928ff16a376 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 @@ -59,11 +59,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -187,7 +187,7 @@ public Result inputSplits( if (fullTableScan) { // scans the partitions and files directly. FileIndex fileIndex = getFileIndex(); - readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths()); + readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { LOG.warn("No partitions found for reading in user provided path."); return Result.EMPTY; @@ -219,7 +219,7 @@ public Result inputSplits( // fallback to full table scan // reading from the earliest, scans the partitions and files directly. FileIndex fileIndex = getFileIndex(); - readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths()); + readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { LOG.warn("No partitions found for reading in user provided path."); return Result.EMPTY; @@ -281,7 +281,7 @@ public Result inputSplits( if (instantRange == null) { // reading from the earliest, scans the partitions and files directly. FileIndex fileIndex = getFileIndex(); - readPartitions = new HashSet<>(fileIndex.getOrBuildPartitionPaths()); + readPartitions = new TreeSet<>(fileIndex.getOrBuildPartitionPaths()); if (readPartitions.size() == 0) { LOG.warn("No partitions found for reading under path: " + path); return Result.EMPTY; @@ -535,7 +535,8 @@ public List filterInstantsWithRange( * * @return the filtered timeline */ - private HoodieTimeline filterInstantsByCondition(HoodieTimeline timeline) { + @VisibleForTesting + public HoodieTimeline filterInstantsByCondition(HoodieTimeline timeline) { final HoodieTimeline oriTimeline = timeline; if (this.skipCompaction) { // the compaction commit uses 'commit' as action which is tricky @@ -601,7 +602,7 @@ public static class Builder { // skip compaction private boolean skipCompaction = false; // skip clustering - private boolean skipClustering = true; + private boolean skipClustering = false; public Builder() { } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java index c8757bf10f37b..9f376f24a296f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/StreamReadMonitoringFunction.java @@ -242,11 +242,9 @@ public void cancel() { if (checkpointLock != null) { // this is to cover the case where cancel() is called before the run() synchronized (checkpointLock) { - issuedInstant = null; isRunning = false; } } else { - issuedInstant = null; isRunning = false; } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java index e110cb6ba9038..e8050d1576183 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java @@ -298,12 +298,27 @@ public class FlinkStreamerConfig extends Configuration { @Parameter(names = {"--sort-columns"}, description = "Columns to sort the data by when clustering.") public String sortColumns = ""; + @Parameter(names = {"--sort-memory"}, description = "Sort memory in MB, default 128MB.") + public Integer sortMemory = 128; + @Parameter(names = {"--max-num-groups"}, description = "Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism. default 30") public Integer maxNumGroups = 30; @Parameter(names = {"--target-partitions"}, description = "Number of partitions to list to create ClusteringPlan, default 2") public Integer targetPartitions = 2; + @Parameter(names = {"--cluster-begin-partition"}, description = "Begin partition used to filter partition (inclusive)") + public String clusterBeginPartition = ""; + + @Parameter(names = {"--cluster-end-partition"}, description = "End partition used to filter partition (inclusive)") + public String clusterEndPartition = ""; + + @Parameter(names = {"--partition-regex-pattern"}, description = "Filter clustering partitions that matched regex pattern") + public String partitionRegexPattern = ""; + + @Parameter(names = {"--partition-selected"}, description = "Partitions to run clustering") + public String partitionSelected = ""; + @Parameter(names = {"--clean-async-enabled"}, description = "Whether to cleanup the old commits immediately on new commits, enabled by default") public Boolean cleanAsyncEnabled = true; @@ -463,7 +478,12 @@ public static org.apache.flink.configuration.Configuration toFlinkConfig(FlinkSt conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, config.targetFileMaxBytes); conf.setLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT, config.smallFileLimit); conf.setInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, config.skipFromLatestPartitions); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_BEGIN_PARTITION, config.clusterBeginPartition); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_END_PARTITION, config.clusterEndPartition); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_REGEX_PATTERN, config.partitionRegexPattern); + conf.setString(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_SELECTED, config.partitionSelected); conf.setString(FlinkOptions.CLUSTERING_SORT_COLUMNS, config.sortColumns); + conf.setInteger(FlinkOptions.WRITE_SORT_MEMORY, config.sortMemory); conf.setInteger(FlinkOptions.CLUSTERING_MAX_NUM_GROUPS, config.maxNumGroups); conf.setInteger(FlinkOptions.CLUSTERING_TARGET_PARTITIONS, config.targetPartitions); conf.setBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED, config.cleanAsyncEnabled); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java index c7a79561b3f75..6d9f666157942 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java @@ -18,9 +18,12 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableConfig; 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.index.HoodieIndex; @@ -30,6 +33,7 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.DataTypeUtils; +import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; @@ -68,12 +72,11 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab @Override public DynamicTableSource createDynamicTableSource(Context context) { Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions()); - ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); - sanityCheck(conf, schema); - setupConfOptions(conf, context.getObjectIdentifier(), context.getCatalogTable(), schema); - Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() -> new ValidationException("Option [path] should not be empty."))); + setupTableOptions(conf.getString(FlinkOptions.PATH), conf); + ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); + setupConfOptions(conf, context.getObjectIdentifier(), context.getCatalogTable(), schema); return new HoodieTableSource( schema, path, @@ -87,12 +90,34 @@ public DynamicTableSink createDynamicTableSink(Context context) { Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions()); checkArgument(!StringUtils.isNullOrEmpty(conf.getString(FlinkOptions.PATH)), "Option [path] should not be empty."); + setupTableOptions(conf.getString(FlinkOptions.PATH), conf); ResolvedSchema schema = context.getCatalogTable().getResolvedSchema(); sanityCheck(conf, schema); setupConfOptions(conf, context.getObjectIdentifier(), context.getCatalogTable(), schema); return new HoodieTableSink(conf, schema); } + /** + * Supplement the table config options if not specified. + */ + private void setupTableOptions(String basePath, Configuration conf) { + StreamerUtil.getTableConfig(basePath, HadoopConfigurations.getHadoopConf(conf)) + .ifPresent(tableConfig -> { + if (tableConfig.contains(HoodieTableConfig.RECORDKEY_FIELDS) + && !conf.contains(FlinkOptions.RECORD_KEY_FIELD)) { + conf.setString(FlinkOptions.RECORD_KEY_FIELD, tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)); + } + if (tableConfig.contains(HoodieTableConfig.PRECOMBINE_FIELD) + && !conf.contains(FlinkOptions.PRECOMBINE_FIELD)) { + conf.setString(FlinkOptions.PRECOMBINE_FIELD, tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)); + } + if (tableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE) + && !conf.contains(FlinkOptions.HIVE_STYLE_PARTITIONING)) { + conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, tableConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)); + } + }); + } + @Override public String factoryIdentifier() { return FACTORY_ID; @@ -119,9 +144,17 @@ public Set> optionalOptions() { * @param schema The table schema */ private void sanityCheck(Configuration conf, ResolvedSchema schema) { - List fields = schema.getColumnNames(); + if (!OptionsResolver.isAppendMode(conf)) { + checkRecordKey(conf, schema); + checkPreCombineKey(conf, schema); + } + } - // validate record key in pk absence. + /** + * Validate the record key. + */ + private void checkRecordKey(Configuration conf, ResolvedSchema schema) { + List fields = schema.getColumnNames(); if (!schema.getPrimaryKey().isPresent()) { String[] recordKeys = conf.get(FlinkOptions.RECORD_KEY_FIELD).split(","); if (recordKeys.length == 1 @@ -139,8 +172,13 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) { + "'" + FlinkOptions.RECORD_KEY_FIELD.key() + "' does not exist in the table schema."); }); } + } - // validate pre_combine key + /** + * Validate pre_combine key. + */ + private void checkPreCombineKey(Configuration conf, ResolvedSchema schema) { + List fields = schema.getColumnNames(); String preCombineField = conf.get(FlinkOptions.PRECOMBINE_FIELD); if (!fields.contains(preCombineField)) { if (OptionsResolver.isDefaultHoodieRecordPayloadClazz(conf)) { @@ -342,7 +380,7 @@ private static void setupWriteOptions(Configuration conf) { private static void inferAvroSchema(Configuration conf, LogicalType rowType) { if (!conf.getOptional(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH).isPresent() && !conf.getOptional(FlinkOptions.SOURCE_AVRO_SCHEMA).isPresent()) { - String inferredSchema = AvroSchemaConverter.convertToSchema(rowType).toString(); + String inferredSchema = AvroSchemaConverter.convertToSchema(rowType, AvroSchemaUtils.getAvroRecordQualifiedName(conf.getString(FlinkOptions.TABLE_NAME))).toString(); conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, inferredSchema); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index fdbf425ed66ca..ab2ec301cc37d 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -67,6 +67,7 @@ import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.source.DynamicTableSource; @@ -153,8 +154,8 @@ public HoodieTableSource( @Nullable Long limit, @Nullable HoodieTableMetaClient metaClient, @Nullable InternalSchemaManager internalSchemaManager) { - this.schema = schema; - this.tableRowType = (RowType) schema.toPhysicalRowDataType().notNull().getLogicalType(); + this.schema = ResolvedSchema.of(schema.getColumns().stream().filter(Column::isPhysical).collect(Collectors.toList())); + this.tableRowType = (RowType) this.schema.toSourceRowDataType().notNull().getLogicalType(); this.path = path; this.partitionKeys = partitionKeys; this.defaultPartName = defaultPartName; 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 374bad86ab2c7..17e3cfa283834 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 @@ -18,16 +18,23 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.client.HoodieFlinkWriteClient; 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.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; +import org.apache.hudi.configuration.OptionsResolver; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.util.AvroSchemaConverter; +import org.apache.hudi.util.DataTypeUtils; import org.apache.hudi.util.FlinkWriteClients; import org.apache.hudi.util.StreamerUtil; @@ -63,8 +70,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; @@ -155,7 +162,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); } @@ -247,11 +254,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( @@ -292,7 +305,9 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable catalogTable, boo if (!resolvedSchema.getPrimaryKey().isPresent()) { throw new CatalogException("Primary key definition is missing"); } - final String avroSchema = AvroSchemaConverter.convertToSchema(resolvedSchema.toPhysicalRowDataType().getLogicalType()).toString(); + final String avroSchema = AvroSchemaConverter.convertToSchema( + resolvedSchema.toPhysicalRowDataType().getLogicalType(), + AvroSchemaUtils.getAvroRecordQualifiedName(tablePath.getObjectName())).toString(); conf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema); // stores two copies of options: @@ -306,6 +321,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); @@ -315,7 +345,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 8644435b5abea..0e1439db50ff9 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 @@ -18,12 +18,14 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.client.HoodieFlinkWriteClient; 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.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -386,9 +388,9 @@ private Table translateSparkTable2Flink(ObjectPath tablePath, Table hiveTable) { if (!parameters.containsKey(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { // 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()); + Option tableConfig = StreamerUtil.getTableConfig(path, hiveConf); + if (tableConfig.isPresent() && tableConfig.get().contains(FlinkOptions.HIVE_STYLE_PARTITIONING.key())) { + hiveStyle = Boolean.parseBoolean(tableConfig.get().getHiveStylePartitioningEnable()); } else { // fallback to the partition path pattern Path hoodieTablePath = new Path(path); @@ -416,11 +418,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); @@ -480,7 +482,9 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig private void initTableIfNotExists(ObjectPath tablePath, CatalogTable catalogTable) { Configuration flinkConf = Configuration.fromMap(catalogTable.getOptions()); - final String avroSchema = AvroSchemaConverter.convertToSchema(catalogTable.getSchema().toPersistedRowDataType().getLogicalType()).toString(); + final String avroSchema = AvroSchemaConverter.convertToSchema( + catalogTable.getSchema().toPersistedRowDataType().getLogicalType(), + AvroSchemaUtils.getAvroRecordQualifiedName(tablePath.getObjectName())).toString(); flinkConf.setString(FlinkOptions.SOURCE_AVRO_SCHEMA, avroSchema); // stores two copies of options: @@ -532,7 +536,7 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table, org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable( tablePath.getDatabaseName(), tablePath.getObjectName()); - hiveTable.setOwner(UserGroupInformation.getCurrentUser().getUserName()); + hiveTable.setOwner(UserGroupInformation.getCurrentUser().getShortUserName()); hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); Map properties = new HashMap<>(table.getOptions()); @@ -596,7 +600,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/InternalSchemaManager.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java index 7fa598bc834a0..3783e642c8d5a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java @@ -93,28 +93,39 @@ public InternalSchema getQuerySchema() { return querySchema; } - InternalSchema getFileSchema(String fileName) { + /** + * Attempts to merge the file and query schema to produce a mergeSchema, prioritising the use of fileSchema types. + * An emptySchema is returned if: + *

    + *
  • 1. An empty querySchema is provided
  • + *
  • 2. querySchema is equal to fileSchema
  • + *
+ * Note that this method returns an emptySchema if merging is not required to be performed. + * @param fileName Name of file to fetch commitTime/versionId for + * @return mergeSchema, i.e. the schema on which the file should be read with + */ + InternalSchema getMergeSchema(String fileName) { if (querySchema.isEmptySchema()) { return querySchema; } long commitInstantTime = Long.parseLong(FSUtils.getCommitTime(fileName)); - InternalSchema fileSchemaUnmerged = InternalSchemaCache.getInternalSchemaByVersionId( + InternalSchema fileSchema = InternalSchemaCache.getInternalSchemaByVersionId( commitInstantTime, tablePath, getHadoopConf(), validCommits); - if (querySchema.equals(fileSchemaUnmerged)) { + if (querySchema.equals(fileSchema)) { return InternalSchema.getEmptyInternalSchema(); } - return new InternalSchemaMerger(fileSchemaUnmerged, querySchema, true, true).mergeSchema(); + return new InternalSchemaMerger(fileSchema, querySchema, true, true).mergeSchema(); } /** - * This method returns a mapping of columns that have type inconsistencies between the fileSchema and querySchema. + * This method returns a mapping of columns that have type inconsistencies between the mergeSchema and querySchema. * This is done by: *
  • 1. Finding the columns with type changes
  • *
  • 2. Get a map storing the index of these columns with type changes; Map of -> (colIdxInQueryFieldNames, colIdxInQuerySchema)
  • *
  • 3. For each selectedField with type changes, build a castMap containing the cast/conversion details; * Map of -> (selectedPos, Cast([from] fileType, [to] queryType))
  • * - * @param fileSchema InternalSchema representation of the file's schema (acquired from commit/.schema metadata) + * @param mergeSchema InternalSchema representation of mergeSchema (prioritise use of fileSchemaType) that is used for reading base parquet files * @param queryFieldNames array containing the columns of a Hudi Flink table * @param queryFieldTypes array containing the field types of the columns of a Hudi Flink table * @param selectedFields array containing the index of the columns of interest required (indexes are based on queryFieldNames and queryFieldTypes) @@ -122,31 +133,33 @@ InternalSchema getFileSchema(String fileName) { * * @see CastMap */ - CastMap getCastMap(InternalSchema fileSchema, String[] queryFieldNames, DataType[] queryFieldTypes, int[] selectedFields) { + CastMap getCastMap(InternalSchema mergeSchema, String[] queryFieldNames, DataType[] queryFieldTypes, int[] selectedFields) { Preconditions.checkArgument(!querySchema.isEmptySchema(), "querySchema cannot be empty"); - Preconditions.checkArgument(!fileSchema.isEmptySchema(), "fileSchema cannot be empty"); + Preconditions.checkArgument(!mergeSchema.isEmptySchema(), "mergeSchema cannot be empty"); CastMap castMap = new CastMap(); // map storing the indexes of columns with type changes Map of -> (colIdxInQueryFieldNames, colIdxInQuerySchema) - Map posProxy = getPosProxy(fileSchema, queryFieldNames); + Map posProxy = getPosProxy(mergeSchema, queryFieldNames); if (posProxy.isEmpty()) { // no type changes castMap.setFileFieldTypes(queryFieldTypes); return castMap; } List selectedFieldList = IntStream.of(selectedFields).boxed().collect(Collectors.toList()); - List fileSchemaAsDataTypes = AvroSchemaConverter.convertToDataType( - AvroInternalSchemaConverter.convert(fileSchema, "tableName")).getChildren(); + // mergeSchema is built with useColumnTypeFromFileSchema = true + List mergeSchemaAsDataTypes = AvroSchemaConverter.convertToDataType( + AvroInternalSchemaConverter.convert(mergeSchema, "tableName")).getChildren(); DataType[] fileFieldTypes = new DataType[queryFieldTypes.length]; for (int i = 0; i < queryFieldTypes.length; i++) { + // position of ChangedType in querySchema Integer posOfChangedType = posProxy.get(i); if (posOfChangedType == null) { // no type change for column; fileFieldType == queryFieldType fileFieldTypes[i] = queryFieldTypes[i]; } else { // type change detected for column; - DataType fileType = fileSchemaAsDataTypes.get(posOfChangedType); - // update fileFieldType match the type found in fileSchema + DataType fileType = mergeSchemaAsDataTypes.get(posOfChangedType); + // update fileFieldType match the type found in mergeSchema fileFieldTypes[i] = fileType; int selectedPos = selectedFieldList.indexOf(i); if (selectedPos != -1) { @@ -162,34 +175,34 @@ CastMap getCastMap(InternalSchema fileSchema, String[] queryFieldNames, DataType /** * For columns that have been modified via the column renaming operation, the column name might be inconsistent - * between querySchema and fileSchema. + * between querySchema and mergeSchema. *

    * As such, this method will identify all columns that have been renamed, and return a string array of column names - * corresponding to the column names found in the fileSchema. + * corresponding to the column names found in the mergeSchema. *

    * This is done by: *

  • 1. Get the rename mapping of -> (colNameFromNewSchema, colNameLastPartFromOldSchema)
  • *
  • 2. For columns that have been renamed, replace them with the old column name
  • * - * @param fileSchema InternalSchema representation of the file's schema (acquired from commit/.schema metadata) + * @param mergeSchema InternalSchema representation of mergeSchema (prioritise use of fileSchemaType) that is used for reading base parquet files * @param queryFieldNames array containing the columns of a Hudi Flink table - * @return String array containing column names corresponding to the column names found in the fileSchema + * @return String array containing column names corresponding to the column names found in the mergeSchema * * @see InternalSchemaUtils#collectRenameCols(InternalSchema, InternalSchema) */ - String[] getFileFieldNames(InternalSchema fileSchema, String[] queryFieldNames) { + String[] getMergeFieldNames(InternalSchema mergeSchema, String[] queryFieldNames) { Preconditions.checkArgument(!querySchema.isEmptySchema(), "querySchema cannot be empty"); - Preconditions.checkArgument(!fileSchema.isEmptySchema(), "fileSchema cannot be empty"); + Preconditions.checkArgument(!mergeSchema.isEmptySchema(), "mergeSchema cannot be empty"); - Map renamedCols = InternalSchemaUtils.collectRenameCols(fileSchema, querySchema); + Map renamedCols = InternalSchemaUtils.collectRenameCols(mergeSchema, querySchema); if (renamedCols.isEmpty()) { return queryFieldNames; } return Arrays.stream(queryFieldNames).map(name -> renamedCols.getOrDefault(name, name)).toArray(String[]::new); } - private Map getPosProxy(InternalSchema fileSchema, String[] queryFieldNames) { - Map> changedCols = InternalSchemaUtils.collectTypeChangedCols(querySchema, fileSchema); + private Map getPosProxy(InternalSchema mergeSchema, String[] queryFieldNames) { + Map> changedCols = InternalSchemaUtils.collectTypeChangedCols(querySchema, mergeSchema); HashMap posProxy = new HashMap<>(changedCols.size()); List fieldNameList = Arrays.asList(queryFieldNames); List columns = querySchema.columns(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/RecordIterators.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/RecordIterators.java index 8657f16ddc974..1bc02bcad4007 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/RecordIterators.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/RecordIterators.java @@ -50,8 +50,8 @@ public static ClosableIterator getParquetRecordIterator( Path path, long splitStart, long splitLength) throws IOException { - InternalSchema fileSchema = internalSchemaManager.getFileSchema(path.getName()); - if (fileSchema.isEmptySchema()) { + InternalSchema mergeSchema = internalSchemaManager.getMergeSchema(path.getName()); + if (mergeSchema.isEmptySchema()) { return new ParquetSplitRecordIterator( ParquetSplitReaderUtil.genPartColumnarRowReader( utcTimestamp, @@ -66,14 +66,14 @@ public static ClosableIterator getParquetRecordIterator( splitStart, splitLength)); } else { - CastMap castMap = internalSchemaManager.getCastMap(fileSchema, fieldNames, fieldTypes, selectedFields); + CastMap castMap = internalSchemaManager.getCastMap(mergeSchema, fieldNames, fieldTypes, selectedFields); Option castProjection = castMap.toRowDataProjection(selectedFields); ClosableIterator itr = new ParquetSplitRecordIterator( ParquetSplitReaderUtil.genPartColumnarRowReader( utcTimestamp, caseSensitive, conf, - internalSchemaManager.getFileFieldNames(fileSchema, fieldNames), // the reconciled field names + internalSchemaManager.getMergeFieldNames(mergeSchema, fieldNames), // the reconciled field names castMap.getFileFieldTypes(), // the reconciled field types partitionSpec, selectedFields, diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java index 4e162d8e2b8c7..8474c2a797ad1 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputFormat.java @@ -173,11 +173,11 @@ private ClosableIterator getRecordIterator( Schema dataSchema = HoodieAvroUtils.removeMetadataFields(new Schema.Parser().parse(tableState.getAvroSchema())); Schema cdcSchema = HoodieCDCUtils.schemaBySupplementalLoggingMode(mode, dataSchema); switch (mode) { - case WITH_BEFORE_AFTER: + case data_before_after: return new BeforeAfterImageIterator(tablePath, tableState, hadoopConf, cdcSchema, fileSplit); - case WITH_BEFORE: + case data_before: return new BeforeImageIterator(conf, hadoopConf, tablePath, tableState, cdcSchema, fileSplit, imageManager); - case OP_KEY: + case op_key_only: return new RecordKeyImageIterator(conf, hadoopConf, tablePath, tableState, cdcSchema, fileSplit, imageManager); default: throw new AssertionError("Unexpected mode" + mode); 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 4ced1bb09fd9d..1e4e507f5481d 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 @@ -193,6 +193,7 @@ protected ClosableIterator initIterator(MergeOnReadInputSplit split) th return new BaseFileOnlyFilteringIterator( split.getInstantRange().get(), this.tableState.getRequiredRowType(), + this.requiredPos, getBaseFileIterator(split.getBasePath().get(), getRequiredPosWithCommitTime(this.requiredPos))); } else { // base file only @@ -228,7 +229,7 @@ protected ClosableIterator initIterator(MergeOnReadInputSplit split) th + "file path: " + split.getBasePath() + "log paths: " + split.getLogPaths() + "hoodie table path: " + split.getTablePath() - + "spark partition Index: " + split.getSplitNumber() + + "flink partition Index: " + split.getSplitNumber() + "merge type: " + split.getMergeType()); } } @@ -548,21 +549,31 @@ static class BaseFileOnlyFilteringIterator implements ClosableIterator private RowData currentRecord; + private int commitTimePos; + BaseFileOnlyFilteringIterator( InstantRange instantRange, RowType requiredRowType, + int[] requiredPos, ClosableIterator nested) { this.nested = nested; this.instantRange = instantRange; - int[] positions = IntStream.range(1, 1 + requiredRowType.getFieldCount()).toArray(); - projection = RowDataProjection.instance(requiredRowType, positions); + this.commitTimePos = getCommitTimePos(requiredPos); + int[] positions; + if (commitTimePos < 0) { + commitTimePos = 0; + positions = IntStream.range(1, 1 + requiredPos.length).toArray(); + } else { + positions = IntStream.range(0, requiredPos.length).toArray(); + } + this.projection = RowDataProjection.instance(requiredRowType, positions); } @Override public boolean hasNext() { while (this.nested.hasNext()) { currentRecord = this.nested.next(); - boolean isInRange = instantRange.isInRange(currentRecord.getString(HOODIE_COMMIT_TIME_COL_POS).toString()); + boolean isInRange = instantRange.isInRange(currentRecord.getString(commitTimePos).toString()); if (isInRange) { return true; } @@ -832,8 +843,13 @@ private Option mergeRowWithLog(RowData curRow, String c GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow); HoodieAvroIndexedRecord hoodieAvroIndexedRecord = new HoodieAvroIndexedRecord(historyAvroRecord); try { - Option resultRecord = recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps).map(Pair::getLeft); - return resultRecord.get().toIndexedRecord(tableSchema, new Properties()); + + + // Option resultRecord = recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps).map(Pair::getLeft); +// return resultRecord.get().toIndexedRecord(tableSchema, new Properties()); + Option> mergeResult = recordMerger.merge(hoodieAvroIndexedRecord, tableSchema, record, tableSchema, payloadProps); + Option resultRecord = mergeResult.map(p -> (HoodieAvroIndexedRecord) p.getLeft()); + return resultRecord; } catch (IOException e) { throw new HoodieIOException("Merge base and delta payloads exception", e); } @@ -898,12 +914,24 @@ public MergeOnReadInputFormat build() { // ------------------------------------------------------------------------- private static int[] getRequiredPosWithCommitTime(int[] requiredPos) { + if (getCommitTimePos(requiredPos) >= 0) { + return requiredPos; + } int[] requiredPos2 = new int[requiredPos.length + 1]; requiredPos2[0] = HOODIE_COMMIT_TIME_COL_POS; System.arraycopy(requiredPos, 0, requiredPos2, 1, requiredPos.length); return requiredPos2; } + private static int getCommitTimePos(int[] requiredPos) { + for (int i = 0; i < requiredPos.length; i++) { + if (requiredPos[i] == HOODIE_COMMIT_TIME_COL_POS) { + return i; + } + } + return -1; + } + @VisibleForTesting public void isEmitDelete(boolean emitDelete) { this.emitDelete = emitDelete; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java index 925819c871e57..9141e473d48de 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroSchemaConverter.java @@ -256,7 +256,7 @@ public static Schema convertToSchema(LogicalType logicalType, String rowName) { throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } Schema timestamp = timestampLogicalType.addToSchema(SchemaBuilder.builder().longType()); return nullable ? nullableSchema(timestamp) : timestamp; @@ -273,7 +273,7 @@ public static Schema convertToSchema(LogicalType logicalType, String rowName) { throw new IllegalArgumentException( "Avro does not support LOCAL TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } Schema localZonedTimestamp = localZonedTimestampLogicalType.addToSchema(SchemaBuilder.builder().longType()); return nullable ? nullableSchema(localZonedTimestamp) : localZonedTimestamp; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java index 5c9988dc0b2ed..38633b8ad9e77 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/AvroToRowDataConverters.java @@ -215,7 +215,7 @@ private static AvroToRowDataConverter createTimestampConverter(int precision) { throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } return avroObject -> { final Instant instant; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ExpressionUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ExpressionUtils.java index 20473acdcda6b..78d374a31af6a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ExpressionUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ExpressionUtils.java @@ -158,9 +158,13 @@ public static Object getValueFromLiteral(ValueLiteralExpression expr) { case BOOLEAN: return expr.getValueAs(Boolean.class).orElse(null); case TINYINT: + return expr.getValueAs(Byte.class).orElse(null); case SMALLINT: + return expr.getValueAs(Short.class).orElse(null); case INTEGER: return expr.getValueAs(Integer.class).orElse(null); + case BIGINT: + return expr.getValueAs(Long.class).orElse(null); case FLOAT: return expr.getValueAs(Float.class).orElse(null); case DOUBLE: diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java index 5d4dac18ae1ab..9f61580da0b49 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/FlinkWriteClients.java @@ -172,6 +172,10 @@ public static HoodieWriteConfig getHoodieClientConfig( .withClusteringTargetFileMaxBytes(conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES)) .withClusteringPlanSmallFileLimit(conf.getLong(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT) * 1024 * 1024L) .withClusteringSkipPartitionsFromLatest(conf.getInteger(FlinkOptions.CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST)) + .withClusteringPartitionFilterBeginPartition(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_BEGIN_PARTITION)) + .withClusteringPartitionFilterEndPartition(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_CLUSTER_END_PARTITION)) + .withClusteringPartitionRegexPattern(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_REGEX_PATTERN)) + .withClusteringPartitionSelected(conf.get(FlinkOptions.CLUSTERING_PLAN_STRATEGY_PARTITION_SELECTED)) .withAsyncClusteringMaxCommits(conf.getInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS)) .build()) .withCleanConfig(HoodieCleanConfig.newBuilder() 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 a6bddf1e82f86..c6eccd3069894 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 @@ -57,8 +57,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; - import java.io.BufferedReader; import java.io.IOException; import java.io.StringReader; @@ -223,7 +221,8 @@ public static boolean tableExists(String basePath, org.apache.hadoop.conf.Config // Hadoop FileSystem FileSystem fs = FSUtils.getFs(basePath, hadoopConf); try { - return fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)); + return fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)) + && fs.exists(new Path(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME), HoodieTableConfig.HOODIE_PROPERTIES_FILE)); } catch (IOException e) { throw new HoodieException("Error while checking whether table exists under path:" + basePath, e); } @@ -287,20 +286,19 @@ public static HoodieTableMetaClient createMetaClient(Configuration conf) { } /** - * Returns the table config or null if the table does not exist. + * Returns the table config or empty if the table does not exist. */ - @Nullable - public static HoodieTableConfig getTableConfig(String basePath, org.apache.hadoop.conf.Configuration hadoopConf) { + public static Option 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, null); + return Option.of(new HoodieTableConfig(fs, metaPath.toString(), null, null)); } } catch (IOException e) { throw new HoodieIOException("Get table config error", e); } - return null; + return Option.empty(); } /** 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 e5d2b315ca78f..ca7adfa286e8c 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 @@ -24,6 +24,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.configuration.OptionsInference; +import org.apache.hudi.exception.SchemaCompatibilityException; import org.apache.hudi.sink.transform.ChainedTransformer; import org.apache.hudi.sink.transform.Transformer; import org.apache.hudi.sink.utils.Pipelines; @@ -41,12 +42,14 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.io.FilePathFilter; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -68,12 +71,15 @@ import java.io.File; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeUnit; +import static org.apache.hudi.config.HoodieWriteConfig.AVRO_SCHEMA_VALIDATE_ENABLE; +import static org.apache.hudi.config.HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP; import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH; import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE; @@ -210,6 +216,16 @@ private void testWriteToHoodie( String jobName, int checkpoints, Map> expected) throws Exception { + testWriteToHoodie(conf, transformer, jobName, checkpoints, true, expected); + } + + private void testWriteToHoodie( + Configuration conf, + Option transformer, + String jobName, + int checkpoints, + boolean restartJob, + Map> expected) throws Exception { StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); execEnv.getConfig().disableObjectReuse(); @@ -217,6 +233,9 @@ private void testWriteToHoodie( // set up checkpoint interval execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE); execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1); + if (!restartJob) { + execEnv.setRestartStrategy(new RestartStrategies.NoRestartStrategyConfiguration()); + } // Read from file source RowType rowType = @@ -515,4 +534,37 @@ public void testHoodiePipelineBuilderSinkWithSchemaSet() throws Exception { execute(execEnv, false, "Api_Sink_Test"); TestData.checkWrittenDataCOW(tempFile, EXPECTED); } + + @Test + public void testColumnDroppingIsNotAllowed() throws Exception { + // Write cols: uuid, name, age, ts, partition + Configuration conf = TestConfigurations.getDefaultConf(tempFile.toURI().toString()); + testWriteToHoodie(conf, "initial write", 1, EXPECTED); + + // Write cols: uuid, name, ts, partition + conf.setBoolean(AVRO_SCHEMA_VALIDATE_ENABLE.key(), false); + conf.setBoolean(SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key(), false); + conf.setString( + FlinkOptions.SOURCE_AVRO_SCHEMA_PATH, + Objects.requireNonNull(Thread.currentThread() + .getContextClassLoader() + .getResource("test_read_schema_dropped_age.avsc") + ).toString() + ); + + // assert job failure with schema compatibility exception + try { + testWriteToHoodie(conf, Option.empty(), "failing job", 1, false, Collections.emptyMap()); + } catch (JobExecutionException e) { + Throwable actualException = e; + while (actualException != null) { + if (actualException.getClass() == SchemaCompatibilityException.class) { + // test is passed + return; + } + actualException = actualException.getCause(); + } + } + throw new AssertionError(String.format("Excepted exception %s is not found", SchemaCompatibilityException.class)); + } } 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 808e9b2d94e36..ab48f421447fb 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 @@ -389,6 +389,44 @@ void testLockForMetadataTable() throws Exception { assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); } + @Test + public void testCommitOnEmptyBatch() throws Exception { + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + conf.setBoolean(HoodieWriteConfig.ALLOW_EMPTY_COMMIT.key(), true); + MockOperatorCoordinatorContext context = new MockOperatorCoordinatorContext(new OperatorID(), 2); + NonThrownExecutor executor = new MockCoordinatorExecutor(context); + try (StreamWriteOperatorCoordinator coordinator = new StreamWriteOperatorCoordinator(conf, context)) { + coordinator.start(); + coordinator.setExecutor(executor); + coordinator.handleEventFromOperator(0, WriteMetadataEvent.emptyBootstrap(0)); + coordinator.handleEventFromOperator(1, WriteMetadataEvent.emptyBootstrap(1)); + + // Coordinator start the instant + String instant = coordinator.getInstant(); + + OperatorEvent event1 = WriteMetadataEvent.builder() + .taskID(0) + .instantTime(instant) + .writeStatus(Collections.emptyList()) + .lastBatch(true) + .build(); + OperatorEvent event2 = WriteMetadataEvent.builder() + .taskID(1) + .instantTime(instant) + .writeStatus(Collections.emptyList()) + .lastBatch(true) + .build(); + coordinator.handleEventFromOperator(0, event1); + coordinator.handleEventFromOperator(1, event2); + + assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1), + "Commit the instant"); + String lastCompleted = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath()); + assertThat("Commits the instant with empty batch anyway", lastCompleted, is(instant)); + assertNull(coordinator.getEventBuffer()[0]); + } + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java index 21dd6fd1d18ab..6a12e0b261976 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.utils.TestWriteBase; import org.apache.hudi.util.FlinkWriteClients; @@ -337,6 +338,42 @@ public void testInsertWithSmallBufferSize() throws Exception { .end(); } + @Test + public void testCommitOnEmptyBatch() throws Exception { + // reset the config option + conf.setBoolean(HoodieWriteConfig.ALLOW_EMPTY_COMMIT.key(), true); + + preparePipeline() + .consume(TestData.DATA_SET_INSERT) + .assertEmptyDataFiles() + .checkpoint(1) + .assertNextEvent() + .checkpointComplete(1) + .checkCompletedInstantCount(1) + // Do checkpoint without data consumption + .checkpoint(2) + .assertNextEvent() + .checkpointComplete(2) + // The instant is committed successfully + .checkCompletedInstantCount(2) + // Continue to consume data + .consume(TestData.DATA_SET_UPDATE_INSERT) + .checkWrittenData(EXPECTED1) + .checkpoint(3) + .assertNextEvent() + .checkpointComplete(3) + .checkCompletedInstantCount(3) + // Commit the data and check after an empty batch + .checkWrittenData(EXPECTED2) + // Do checkpoint without data consumption + .checkpoint(4) + .assertNextEvent() + .checkpointComplete(4) + .checkCompletedInstantCount(4) + .checkWrittenData(EXPECTED2) + .end(); + } + protected Map getMiniBatchExpected() { Map expected = new HashMap<>(); // the last 2 lines are merged 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..f7ad6bf5acc7f --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java @@ -0,0 +1,184 @@ +/* + * 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; + +/** + * 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().size()); + + // rollback path structure: tablePath/.hoodie/.temp/${commitInstant}/${partition}/${fileGroup}_${fileInstant}.parquet.marker.APPEND + HoodieInstant instant = activeCompletedTimeline.getInstants().get(0); + String commitInstant = instant.getTimestamp(); + String filename = activeCompletedTimeline.getInstants().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..3e59df8498062 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,44 @@ void testDateBasedKeyGenerator(String partitionFormat) { assertThat(keyGen2.getPartitionPath(rowData2), is("dt=" + expectedPartition2)); assertThat(keyGen2.getPartitionPath(rowData3), is("dt=" + expectedPartition3)); } + + @Test + void testPrimaryKeylessWrite() { + Configuration conf = TestConfigurations.getDefaultConf("path1"); + conf.setString(FlinkOptions.RECORD_KEY_FIELD, ""); + final RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23, + TimestampData.fromEpochMillis(1), StringData.fromString("par1")); + final RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE); + assertThat(keyGen1.getRecordKey(rowData1), is("__empty__")); + + // null record key and partition path + final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, StringData.fromString("Danny"), 23, + TimestampData.fromEpochMillis(1), null); + assertThat(keyGen1.getRecordKey(rowData2), is("__empty__")); + + // empty record key and partition path + final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString("Danny"), 23, + TimestampData.fromEpochMillis(1), StringData.fromString("")); + assertThat(keyGen1.getRecordKey(rowData3), is("__empty__")); + } + + @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 29f280e61208b..ec671bdd68428 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 @@ -18,6 +18,7 @@ package org.apache.hudi.sink.cluster; +import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.common.model.WriteOperationType; @@ -50,8 +51,12 @@ 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.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; @@ -63,10 +68,14 @@ import java.io.File; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -113,7 +122,9 @@ public void testHoodieFlinkClustering() throws Exception { FlinkClusteringConfig cfg = new FlinkClusteringConfig(); cfg.path = tempFile.getAbsolutePath(); cfg.targetPartitions = 4; + cfg.sortMemory = 256; Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg); + assertEquals(256, conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY)); // create metaClient HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf); @@ -290,4 +301,309 @@ 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 firstClusteringInstant = HoodieActiveTimeline.createNewInstantTime(); + + HoodieFlinkWriteClient writeClient = FlinkWriteClients.createWriteClient(conf); + HoodieFlinkTable table = writeClient.getHoodieTable(); + + boolean scheduled = writeClient.scheduleClusteringAtInstant(firstClusteringInstant, 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(firstClusteringInstant); + 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(firstClusteringInstant, 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); + + HoodieInstant secondClusteringInstant = timeline.lastInstant().get(); + List inputFileGroups = ClusteringUtils.getClusteringPlan(table.getMetaClient(), secondClusteringInstant).get().getRight().getInputGroups(); + // clustering plan has no previous file slice generated by previous pending clustering + assertFalse(inputFileGroups + .stream().anyMatch(fg -> fg.getSlices() + .stream().anyMatch(s -> s.getDataFilePath().contains(firstClusteringInstant)))); + } + + /** + * Test to ensure that creating a table with a column of TIMESTAMP(9) will throw errors + * @throws Exception + */ + @Test + public void testHoodieFlinkClusteringWithTimestampNanos() { + // 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, 4); + Map options = new HashMap<>(); + 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"); + + // row schema + final DataType dataType = DataTypes.ROW( + DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key + DataTypes.FIELD("name", DataTypes.VARCHAR(10)), + DataTypes.FIELD("age", DataTypes.INT()), + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(9)), // precombine field + DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) + .notNull(); + + final RowType rowType = (RowType) dataType.getLogicalType(); + final List fields = rowType.getFields().stream() + .map(RowType.RowField::asSummaryString).collect(Collectors.toList()); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL( + "t1", fields, options, true, "uuid", "partition"); + TableResult tableResult = tableEnv.executeSql(hoodieTableDDL); + + // insert rows with timestamp of microseconds precision; timestamp(6) + final String insertSql = "insert into t1 values\n" + + "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01.100001001','par1'),\n" + + "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02.100001001','par1'),\n" + + "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03.100001001','par2'),\n" + + "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04.100001001','par2'),\n" + + "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05.100001001','par3'),\n" + + "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06.100001001','par3'),\n" + + "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07.100001001','par4'),\n" + + "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08.100001001','par4')"; + + assertThrows(ValidationException.class, () -> tableEnv.executeSql(insertSql), + "Avro does not support TIMESTAMP type with precision: 9, it only support precisions <= 6."); + } + + @Test + public void testHoodieFlinkClusteringWithTimestampMicros() 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, 4); + Map options = new HashMap<>(); + options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath()); + + // use append mode + options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value()); + + // row schema + final DataType dataType = DataTypes.ROW( + DataTypes.FIELD("uuid", DataTypes.VARCHAR(20)),// record key + DataTypes.FIELD("name", DataTypes.VARCHAR(10)), + DataTypes.FIELD("age", DataTypes.INT()), + DataTypes.FIELD("ts", DataTypes.TIMESTAMP(6)), // precombine field + DataTypes.FIELD("partition", DataTypes.VARCHAR(10))) + .notNull(); + final RowType rowType = (RowType) dataType.getLogicalType(); + final List fields = rowType.getFields().stream() + .map(RowType.RowField::asSummaryString).collect(Collectors.toList()); + + String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL( + "t1", fields, options, true, "uuid", "partition"); + tableEnv.executeSql(hoodieTableDDL); + + // insert rows with timestamp of microseconds precision; timestamp(6) + final String insertSql = "insert into t1 values\n" + + "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01.100001','par1'),\n" + + "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02.100001','par1'),\n" + + "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03.100001','par2'),\n" + + "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04.100001','par2'),\n" + + "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05.100001','par3'),\n" + + "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06.100001','par3'),\n" + + "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07.100001','par4'),\n" + + "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08.100001','par4')"; + tableEnv.executeSql(insertSql).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 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(instant -> instant.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()); + + 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); + + dataStream + .addSink(new ClusteringCommitSink(conf)) + .name("clustering_commit") + .uid("uid_clustering_commit") + .setParallelism(1); + + env.execute("flink_hudi_clustering"); + + // test output + final Map expected = new HashMap<>(); + expected.put("par1", "[id1,par1,id1,Danny,23,1100001,par1, id2,par1,id2,Stephen,33,2100001,par1]"); + expected.put("par2", "[id3,par2,id3,Julian,53,3100001,par2, id4,par2,id4,Fabian,31,4100001,par2]"); + expected.put("par3", "[id5,par3,id5,Sophia,18,5100001,par3, id6,par3,id6,Emma,20,6100001,par3]"); + expected.put("par4", "[id7,par4,id7,Bob,44,7100001,par4, id8,par4,id8,Han,56,8100001,par4]"); + TestData.checkWrittenData(tempFile, expected, 4); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCustomSerDe.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCustomSerDe.java new file mode 100644 index 0000000000000..89962bf834785 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCustomSerDe.java @@ -0,0 +1,81 @@ +/* + * 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.compact; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericFixed; +import org.apache.hudi.common.model.EventTimeAvroPayload; +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.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.util.collection.BitCaskDiskMap; +import org.apache.hudi.common.util.collection.RocksDbDiskMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +/** + * Tests for custom SerDe of non-primitive avro types when using Avro versions > 1.10.0. + * The avro version used by hudi-flink module is 1.10.0, these tests are placed here so that avro 1.10.0 is used, + * allowing the error caused by anonymous classes to be thrown. + */ +public class TestCustomSerDe extends HoodieCommonTestHarness { + + @BeforeEach + public void setup() { + initPath(); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBitCaskDiskMapPutDecimal(boolean isCompressionEnabled) throws IOException { + BitCaskDiskMap bitCaskDiskMap = new BitCaskDiskMap<>(basePath, isCompressionEnabled); + HoodieRecord avroRecord = createAvroRecordWithDecimalOrderingField(); + bitCaskDiskMap.put(avroRecord.getRecordKey(), avroRecord); + assertDoesNotThrow(() -> bitCaskDiskMap.get(avroRecord.getRecordKey())); + } + + @Test + public void testRocksDbDiskMapPutDecimal() throws IOException { + RocksDbDiskMap rocksDbBasedMap = new RocksDbDiskMap<>(basePath); + HoodieRecord avroRecord = createAvroRecordWithDecimalOrderingField(); + rocksDbBasedMap.put(avroRecord.getRecordKey(), avroRecord); + assertDoesNotThrow(() -> rocksDbBasedMap.get(avroRecord.getRecordKey())); + } + + private static HoodieRecord createAvroRecordWithDecimalOrderingField() { + Schema precombineFieldSchema = LogicalTypes.decimal(20, 0) + .addToSchema(Schema.createFixed("fixed", null, "record.precombineField", 9)); + byte[] decimalFieldBytes = new byte[] {0, 0, 0, 1, -122, -16, -116, -90, -32}; + GenericFixed genericFixed = new GenericData.Fixed(precombineFieldSchema, decimalFieldBytes); + + // nullifying the record attribute in EventTimeAvroPayload here as it is not required in the test + return new HoodieAvroRecord<>(new HoodieKey("recordKey", "partitionPath"), + new EventTimeAvroPayload(null, (Comparable) genericFixed)); + } + +} diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java index fe7ce3f9478d6..041cd12c97a6b 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java @@ -30,11 +30,14 @@ import org.junit.jupiter.api.io.TempDir; import java.io.File; +import java.util.Arrays; +import java.util.Collections; import java.util.stream.IntStream; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNull; /** * Test cases for {@link CkpMetadata}. @@ -74,4 +77,28 @@ void testWriteAndReadMessage() { metadata.abortInstant("7"); assertThat(metadata.getMessages().size(), is(5)); } + + @Test + void testBootstrap() throws Exception { + CkpMetadata metadata = getCkpMetadata(""); + // write 4 instants to the ckp_meta + IntStream.range(0, 4).forEach(i -> metadata.startInstant(i + "")); + assertThat("The first instant should be removed from the instant cache", + metadata.getInstantCache(), is(Arrays.asList("1", "2", "3"))); + + // simulate the reboot of coordinator + CkpMetadata metadata1 = getCkpMetadata(""); + metadata1.bootstrap(); + assertNull(metadata1.getInstantCache(), "The instant cache should be recovered from bootstrap"); + + metadata1.startInstant("4"); + assertThat("The first instant should be removed from the instant cache", + metadata1.getInstantCache(), is(Collections.singletonList("4"))); + } + + private CkpMetadata getCkpMetadata(String uniqueId) { + String basePath = tempFile.getAbsolutePath(); + FileSystem fs = FSUtils.getFs(basePath, HadoopConfigurations.getHadoopConf(new Configuration())); + return CkpMetadata.getInstance(fs, basePath); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java index b6ae0767d68a3..b741aab83d588 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java @@ -413,5 +413,11 @@ protected String lastCompleteInstant() { ? TestUtils.getLastDeltaCompleteInstant(basePath) : TestUtils.getLastCompleteInstant(basePath, HoodieTimeline.COMMIT_ACTION); } + + public TestHarness checkCompletedInstantCount(int count) { + boolean isMor = OptionsResolver.isMorTable(conf); + assertEquals(count, TestUtils.getCompletedInstantCount(basePath, isMor ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION)); + return this; + } } } 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 a37d69e7026fe..fd04c024cf58a 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,12 +18,17 @@ 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; +import org.apache.hudi.utils.TestData; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; @@ -31,12 +36,17 @@ 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; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test cases for {@link IncrementalInputSplits}. @@ -53,10 +63,12 @@ private void init() throws IOException { void testFilterInstantsWithRange() { HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient, true); Configuration conf = TestConfigurations.getDefaultConf(basePath); + conf.set(FlinkOptions.READ_STREAMING_SKIP_CLUSTERING, true); IncrementalInputSplits iis = IncrementalInputSplits.builder() .conf(conf) .path(new Path(basePath)) .rowType(TestConfigurations.ROW_TYPE) + .skipClustering(conf.getBoolean(FlinkOptions.READ_STREAMING_SKIP_CLUSTERING)) .build(); HoodieInstant commit1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); @@ -93,4 +105,72 @@ void testFilterInstantsWithRange() { assertEquals(3, instantRange4.size()); } + @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().size()); + } + + @Test + void testInputSplitsSortedByPartition() throws Exception { + HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient, true); + Configuration conf = TestConfigurations.getDefaultConf(basePath); + // To enable a full table scan + conf.set(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST); + TestData.writeData(TestData.DATA_SET_INSERT, conf); + TestData.writeData(TestData.DATA_SET_INSERT_SEPARATE_PARTITION, conf); + IncrementalInputSplits iis = IncrementalInputSplits.builder() + .conf(conf) + .path(new Path(basePath)) + .rowType(TestConfigurations.ROW_TYPE) + .build(); + + // File slices use the current timestamp as the baseInstantTime (e.g. "20230320110000000"), so choose an end timestamp + // such that the current timestamp <= end timestamp based on string comparison + HoodieInstant commit = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3000"); + timeline.createNewInstant(commit); + + IncrementalInputSplits.Result result = iis.inputSplits(metaClient, metaClient.getHadoopConf(), null, false); + + List partitions = result.getInputSplits().stream().map(split -> { + Option basePath = split.getBasePath(); + assertTrue(basePath.isPresent()); + + // The partition is the parent directory of the data file + String[] pathParts = basePath.get().split("/"); + assertTrue(pathParts.length >= 2); + return pathParts[pathParts.length - 2]; + }).collect(Collectors.toList()); + + assertEquals(Arrays.asList("par1", "par2", "par3", "par4", "par5", "par6"), partitions); + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java index 541890f7b05f4..616edc37f1cda 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java @@ -252,6 +252,71 @@ public void testCheckpointRestore() throws Exception { } } + /** + * When stopping with savepoint, these interface methods are called: + *
      + *
    • cancel()
    • + *
    • snapshotState()
    • + *
    • close()
    • + *
    + * This test ensured that the state is saved properly when these 3 methods are called in the order listed above. + */ + @Test + public void testStopWithSavepointAndRestore() throws Exception { + TestData.writeData(TestData.DATA_SET_INSERT, conf); + conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST); + StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf); + OperatorSubtaskState state; + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + CountDownLatch latch = new CountDownLatch(4); + CollectingSourceContext sourceContext = new CollectingSourceContext(latch); + runAsync(sourceContext, function); + + assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation"); + Thread.sleep(1000L); + + // Simulate a stop-with-savepoint + function.cancel(); + + state = harness.snapshot(1, 1); + + // Stop the stream task. + function.close(); + + assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation"); + assertThat("Should produce the expected splits", + sourceContext.getPartitionPaths(), is("par1,par2,par3,par4")); + assertTrue(sourceContext.splits.stream().noneMatch(split -> split.getInstantRange().isPresent()), + "All instants should have range limit"); + + } + + TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf); + StreamReadMonitoringFunction function2 = TestUtils.getMonitorFunc(conf); + try (AbstractStreamOperatorTestHarness harness = createHarness(function2)) { + harness.setup(); + // Recover to process the remaining snapshots. + harness.initializeState(state); + harness.open(); + + CountDownLatch latch = new CountDownLatch(4); + CollectingSourceContext sourceContext = new CollectingSourceContext(latch); + runAsync(sourceContext, function2); + + // Stop the stream task. + function.close(); + + assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation"); + assertThat("Should produce the expected splits", + sourceContext.getPartitionPaths(), is("par1,par2,par3,par4")); + assertTrue(sourceContext.splits.stream().allMatch(split -> split.getInstantRange().isPresent()), + "All the instants should have range limit"); + } + } + private AbstractStreamOperatorTestHarness createHarness( StreamReadMonitoringFunction function) throws Exception { StreamSource streamSource = new StreamSource<>(function); 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 10f2bcd095a42..0a80b9168c326 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 @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.table.catalog.HoodieCatalogTestUtils; import org.apache.hudi.table.catalog.HoodieHiveCatalog; @@ -162,7 +163,7 @@ void testStreamReadFromSpecifiedCommitWithChangelog(HoodieCDCSupplementalLogging .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) .option(FlinkOptions.READ_AS_STREAMING, true) .option(FlinkOptions.CDC_ENABLED, true) - .option(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE, mode.getValue()) + .option(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE, mode.name()) .end(); streamTableEnv.executeSql(hoodieTableDDL); String insertInto = "insert into t1 select * from source"; @@ -359,6 +360,36 @@ void testAppendWriteReadSkippingClustering() throws Exception { assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT); } + @Test + void testAppendWriteWithClusteringBatchRead() throws Exception { + // create filesystem table named source + String createSource = TestConfigurations.getFileSourceDDL("source", 4); + streamTableEnv.executeSql(createSource); + + String hoodieTableDDL = sql("t1") + .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) + .option(FlinkOptions.OPERATION, "insert") + .option(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED,true) + .option(FlinkOptions.CLUSTERING_ASYNC_ENABLED, true) + .option(FlinkOptions.CLUSTERING_DELTA_COMMITS,2) + .option(FlinkOptions.CLUSTERING_TASKS, 1) + .option(FlinkOptions.CLEAN_RETAIN_COMMITS, 1) + .end(); + streamTableEnv.executeSql(hoodieTableDDL); + String insertInto = "insert into t1 select * from source"; + execInsertSql(streamTableEnv, insertInto); + + streamTableEnv.getConfig().getConfiguration() + .setBoolean("table.dynamic-table-options.enabled", true); + final String query = String.format("select * from t1/*+ options('read.start-commit'='%s')*/", + FlinkOptions.START_COMMIT_EARLIEST); + + List rows = CollectionUtil.iterableToList(() -> streamTableEnv.sqlQuery(query).execute().collect()); + // batch read will not lose data when cleaned clustered files. + assertRowsEquals(rows, CollectionUtils.combine(TestData.DATA_SET_SOURCE_INSERT_FIRST_COMMIT, + TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT)); + } + @Test void testStreamWriteWithCleaning() { // create filesystem table named source @@ -678,14 +709,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); @@ -1636,6 +1669,34 @@ void testWriteReadWithComputedColumns() { assertRowsEquals(result2, "[+I[3]]"); } + @Test + void testWriteReadWithComputedColumnsInTheMiddle() { + TableEnvironment tableEnv = batchTableEnv; + String createTable = sql("t1") + .field("f0 int") + .field("f1 int") + .field("f2 as f0 + f1") + .field("f3 varchar(10)") + .option(FlinkOptions.PATH, tempFile.getAbsolutePath()) + .option(FlinkOptions.PRECOMBINE_FIELD, "f1") + .pkField("f0") + .noPartition() + .end(); + tableEnv.executeSql(createTable); + + String insertInto = "insert into t1(f0, f1, f3) values\n" + + "(1, 2, 'abc')"; + execInsertSql(tableEnv, insertInto); + + List result1 = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select * from t1").execute().collect()); + assertRowsEquals(result1, "[+I[1, 2, 3, abc]]"); + + List result2 = CollectionUtil.iterableToList( + () -> tableEnv.sqlQuery("select f2 from t1").execute().collect()); + assertRowsEquals(result2, "[+I[3]]"); + } + @ParameterizedTest @EnumSource(value = HoodieTableType.class) void testWriteReadWithLocalTimestamp(HoodieTableType tableType) { @@ -1667,6 +1728,135 @@ void testWriteReadWithLocalTimestamp(HoodieTableType tableType) { assertRowsEquals(result, expected); } + @ParameterizedTest + @MethodSource("tableTypeQueryTypeNumInsertAndCompactionDeltaCommitsParams") + void testReadMetaFields(HoodieTableType tableType, String queryType, int numInsertBatches, int compactionDeltaCommits) throws Exception { + String path = tempFile.getAbsolutePath(); + String hoodieTableDDL = sql("t1") + .field("id int") + .field("name varchar(10)") + .field("ts timestamp(6)") + .field("`partition` varchar(10)") + .pkField("id") + .partitionField("partition") + .option(FlinkOptions.TABLE_TYPE, tableType) + .option(FlinkOptions.QUERY_TYPE, queryType) + .option(FlinkOptions.COMPACTION_ASYNC_ENABLED, true) + .option(FlinkOptions.COMPACTION_DELTA_COMMITS, compactionDeltaCommits) + .option(FlinkOptions.PATH, path) + .end(); + batchTableEnv.executeSql(hoodieTableDDL); + + final String[] insertInto = new String[] { + "insert into t1 values(1,'Danny',TIMESTAMP '2021-12-01 01:02:01.100001', 'par1')", + "insert into t1 values(2,'Stephen',TIMESTAMP '2021-12-02 03:04:02.200002', 'par2')", + "insert into t1 values(3,'Julian',TIMESTAMP '2021-12-03 13:14:03.300003', 'par3')"}; + + // Queries without meta fields. + String[] template1 = new String[] { + "+I[1, Danny, 2021-12-01T01:02:01.100001, par1]", + ", +I[2, Stephen, 2021-12-02T03:04:02.200002, par2]", + ", +I[3, Julian, 2021-12-03T13:14:03.300003, par3]" + }; + + // Meta field '_hoodie_commit_time' in the first position. + String[] template2 = new String[] { + "+I[%s, 1, par1, 1, Danny, 2021-12-01T01:02:01.100001, par1]", + ", +I[%s, 2, par2, 2, Stephen, 2021-12-02T03:04:02.200002, par2]", + ", +I[%s, 3, par3, 3, Julian, 2021-12-03T13:14:03.300003, par3]" + }; + + // Meta fields at random positions. + String[] template3 = new String[] { + "+I[1, %s, Danny, 1, 2021-12-01T01:02:01.100001, par1, par1]", + ", +I[2, %s, Stephen, 2, 2021-12-02T03:04:02.200002, par2, par2]", + ", +I[3, %s, Julian, 3, 2021-12-03T13:14:03.300003, par3, par3]" + }; + + StringBuilder expected1 = new StringBuilder(); + StringBuilder expected2 = new StringBuilder(); + StringBuilder expected3 = new StringBuilder(); + + expected1.append("["); + expected2.append("["); + expected3.append("["); + for (int i = 0; i < numInsertBatches; i++) { + execInsertSql(batchTableEnv, insertInto[i]); + String commitTime = tableType.equals(HoodieTableType.MERGE_ON_READ) + ? TestUtils.getLastDeltaCompleteInstant(path) : TestUtils.getLastCompleteInstant(path); + expected1.append(template1[i]); + expected2.append(String.format(template2[i], commitTime)); + expected3.append(String.format(template3[i], commitTime)); + } + expected1.append("]"); + expected2.append("]"); + expected3.append("]"); + String readHoodieTableDDL; + batchTableEnv.executeSql("drop table t1"); + readHoodieTableDDL = sql("t1") + .field("id int") + .field("name varchar(10)") + .field("ts timestamp(6)") + .field("`partition` varchar(10)") + .pkField("id") + .partitionField("partition") + .option(FlinkOptions.TABLE_TYPE, tableType) + .option(FlinkOptions.QUERY_TYPE, queryType) + .option(FlinkOptions.COMPACTION_ASYNC_ENABLED, true) + .option(FlinkOptions.COMPACTION_DELTA_COMMITS, compactionDeltaCommits) + .option(FlinkOptions.PATH, path) + .end(); + batchTableEnv.executeSql(readHoodieTableDDL); + + List result = execSelectSql(batchTableEnv, "select * from t1", ExecMode.BATCH); + assertRowsEquals(result, expected1.toString()); + + batchTableEnv.executeSql("drop table t1"); + readHoodieTableDDL = sql("t1") + .field("_hoodie_commit_time string") + .field("_hoodie_record_key string") + .field("_hoodie_partition_path string") + .field("id int") + .field("name varchar(10)") + .field("ts timestamp(6)") + .field("`partition` varchar(10)") + .pkField("id") + .partitionField("partition") + .option(FlinkOptions.TABLE_TYPE, tableType) + .option(FlinkOptions.QUERY_TYPE, queryType) + .option(FlinkOptions.COMPACTION_ASYNC_ENABLED, true) + .option(FlinkOptions.COMPACTION_DELTA_COMMITS, compactionDeltaCommits) + .option(FlinkOptions.PATH, path) + .end(); + batchTableEnv.executeSql(readHoodieTableDDL); + + result = execSelectSql(batchTableEnv, "select * from t1", ExecMode.BATCH); + assertRowsEquals(result, expected2.toString()); + + batchTableEnv.executeSql("drop table t1"); + readHoodieTableDDL = sql("t1") + .field("id int") + .field("_hoodie_commit_time string") + .field("name varchar(10)") + .field("_hoodie_record_key string") + .field("ts timestamp(6)") + .field("_hoodie_partition_path string") + .field("`partition` varchar(10)") + .pkField("id") + .partitionField("partition") + .option(FlinkOptions.TABLE_TYPE, tableType) + .option(FlinkOptions.QUERY_TYPE, queryType) + .option(FlinkOptions.COMPACTION_ASYNC_ENABLED, true) + .option(FlinkOptions.COMPACTION_DELTA_COMMITS, compactionDeltaCommits) + .option(FlinkOptions.PATH, path) + .end(); + batchTableEnv.executeSql(readHoodieTableDDL); + + result = execSelectSql(batchTableEnv, "select * from t1", ExecMode.BATCH); + assertRowsEquals(result, expected3.toString()); + + } + // ------------------------------------------------------------------------- // Utilities // ------------------------------------------------------------------------- @@ -1713,6 +1903,19 @@ private static Stream tableTypeAndPartitioningParams() { return Stream.of(data).map(Arguments::of); } + /** + * Return test params => (HoodieTableType, query type, num insert batches, num compaction delta commits). + */ + private static Stream tableTypeQueryTypeNumInsertAndCompactionDeltaCommitsParams() { + return Arrays.stream(new Object[][] { + {HoodieTableType.COPY_ON_WRITE, FlinkOptions.QUERY_TYPE_INCREMENTAL, 1, 1}, + {HoodieTableType.COPY_ON_WRITE, FlinkOptions.QUERY_TYPE_READ_OPTIMIZED, 1, 1}, + {HoodieTableType.MERGE_ON_READ, FlinkOptions.QUERY_TYPE_SNAPSHOT, 1, 1}, + {HoodieTableType.MERGE_ON_READ, FlinkOptions.QUERY_TYPE_SNAPSHOT, 1, 3}, + {HoodieTableType.MERGE_ON_READ, FlinkOptions.QUERY_TYPE_SNAPSHOT, 3, 2} + }).map(Arguments::of); + } + /** * Return test params => (index type, hive style partitioning). */ diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java index f7a35e57f2b09..199327a14a0c1 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java @@ -18,6 +18,7 @@ package org.apache.hudi.table; +import org.apache.hudi.avro.AvroSchemaUtils; import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.EventTimeAvroPayload; import org.apache.hudi.configuration.FlinkOptions; @@ -28,6 +29,7 @@ import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator; import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.SchemaBuilder; import org.apache.hudi.utils.TestConfigurations; @@ -64,7 +66,8 @@ public class TestHoodieTableFactory { private static final String AVRO_SCHEMA_FILE_PATH = Objects.requireNonNull(Thread.currentThread() .getContextClassLoader().getResource("test_read_schema.avsc")).toString(); private static final String INFERRED_SCHEMA = "{\"type\":\"record\"," - + "\"name\":\"record\"," + + "\"name\":\"t1_record\"," + + "\"namespace\":\"hoodie.t1\"," + "\"fields\":[" + "{\"name\":\"uuid\",\"type\":[\"null\",\"string\"],\"default\":null}," + "{\"name\":\"name\",\"type\":[\"null\",\"string\"],\"default\":null}," @@ -86,17 +89,26 @@ void beforeEach() throws IOException { } @Test - void testRequiredOptionsForSource() { - // miss pk and precombine key will throw exception + void testRequiredOptions() { ResolvedSchema schema1 = SchemaBuilder.instance() .field("f0", DataTypes.INT().notNull()) .field("f1", DataTypes.VARCHAR(20)) .field("f2", DataTypes.TIMESTAMP(3)) .build(); final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2"); - assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext1)); + + // createDynamicTableSource doesn't call sanity check, will not throw exception + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext1)); + // miss pk and precombine key will throw exception when create sink assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext1)); + // append mode does not throw + this.conf.set(FlinkOptions.OPERATION, "insert"); + final MockContext sourceContext11 = MockContext.getInstance(this.conf, schema1, "f2"); + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext11)); + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext11)); + this.conf.set(FlinkOptions.OPERATION, FlinkOptions.OPERATION.defaultValue()); + // a non-exists precombine key will throw exception ResolvedSchema schema2 = SchemaBuilder.instance() .field("f0", DataTypes.INT().notNull()) @@ -105,7 +117,8 @@ void testRequiredOptionsForSource() { .build(); this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, "non_exist_field"); final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2"); - assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext2)); + // createDynamicTableSource doesn't call sanity check, will not throw exception + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext2)); assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext2)); this.conf.setString(FlinkOptions.PRECOMBINE_FIELD, FlinkOptions.PRECOMBINE_FIELD.defaultValue()); @@ -120,17 +133,17 @@ void testRequiredOptionsForSource() { HoodieTableSource tableSource = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3); HoodieTableSink tableSink = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext3); // the precombine field is overwritten - assertThat(tableSource.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); assertThat(tableSink.getConf().getString(FlinkOptions.PRECOMBINE_FIELD), is(FlinkOptions.NO_PRE_COMBINE)); // precombine field not specified, use the default payload clazz assertThat(tableSource.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); assertThat(tableSink.getConf().getString(FlinkOptions.PAYLOAD_CLASS_NAME), is(FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue())); - // given pk but miss the pre combine key with DefaultHoodieRecordPayload should throw this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, DefaultHoodieRecordPayload.class.getName()); final MockContext sourceContext4 = MockContext.getInstance(this.conf, schema3, "f2"); - assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSource(sourceContext4)); + // createDynamicTableSource doesn't call sanity check, will not throw exception + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext4)); + // given pk but miss the pre combine key with DefaultHoodieRecordPayload should throw assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext4)); this.conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, FlinkOptions.PAYLOAD_CLASS_NAME.defaultValue()); @@ -167,6 +180,74 @@ void testRequiredOptionsForSource() { assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSink(sourceContext6)); } + @Test + void testSupplementTableConfig() throws Exception { + String tablePath = new File(tempFile.getAbsolutePath(), "dummy").getAbsolutePath(); + // add pk and pre-combine key to table config + Configuration tableConf = new Configuration(); + tableConf.setString(FlinkOptions.PATH, tablePath); + tableConf.setString(FlinkOptions.TABLE_NAME, "t2"); + tableConf.setString(FlinkOptions.RECORD_KEY_FIELD, "f0,f1"); + tableConf.setString(FlinkOptions.PRECOMBINE_FIELD, "f2"); + + StreamerUtil.initTableIfNotExists(tableConf); + + Configuration writeConf = new Configuration(); + writeConf.set(FlinkOptions.PATH, tablePath); + writeConf.set(FlinkOptions.TABLE_NAME, "t2"); + + // fallback to table config + ResolvedSchema schema1 = SchemaBuilder.instance() + .field("f0", DataTypes.INT().notNull()) + .field("f1", DataTypes.VARCHAR(20)) + .field("f2", DataTypes.TIMESTAMP(3)) + .field("ts", DataTypes.TIMESTAMP(3)) + .build(); + final MockContext sourceContext1 = MockContext.getInstance(writeConf, schema1, "f2"); + HoodieTableSource source1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1); + HoodieTableSink sink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext1); + assertThat("pk not provided, fallback to table config", + source1.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1")); + assertThat("pk not provided, fallback to table config", + sink1.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1")); + assertThat("pre-combine key not provided, fallback to table config", + source1.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f2")); + assertThat("pre-combine key not provided, fallback to table config", + sink1.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f2")); + + // write config always has higher priority + // set up a different primary key and pre_combine key with table config options + writeConf.setString(FlinkOptions.RECORD_KEY_FIELD, "f0"); + writeConf.setString(FlinkOptions.PRECOMBINE_FIELD, "f1"); + + final MockContext sourceContext2 = MockContext.getInstance(writeConf, schema1, "f2"); + HoodieTableSource source2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2); + HoodieTableSink sink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sourceContext2); + assertThat("choose pk from write config", + source2.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0")); + assertThat("choose pk from write config", + sink2.getConf().get(FlinkOptions.RECORD_KEY_FIELD), is("f0")); + assertThat("choose preCombine key from write config", + source2.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f1")); + assertThat("choose preCombine pk from write config", + sink2.getConf().get(FlinkOptions.PRECOMBINE_FIELD), is("f1")); + + writeConf.removeConfig(FlinkOptions.RECORD_KEY_FIELD); + writeConf.removeConfig(FlinkOptions.PRECOMBINE_FIELD); + + // pk defined in table config but missing in schema will throw + ResolvedSchema schema2 = SchemaBuilder.instance() + .field("f1", DataTypes.VARCHAR(20)) + .field("f2", DataTypes.TIMESTAMP(3)) + .field("ts", DataTypes.TIMESTAMP(3)) + .build(); + final MockContext sourceContext3 = MockContext.getInstance(writeConf, schema2, "f2"); + assertDoesNotThrow(() -> new HoodieTableFactory().createDynamicTableSource(sourceContext3), + "createDynamicTableSource won't call sanity check"); + assertThrows(HoodieValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sourceContext3), + "f0 is in table config as record key, but missing in input schema"); + } + @Test void testInferAvroSchemaForSource() { // infer the schema if not specified @@ -181,6 +262,21 @@ void testInferAvroSchemaForSource() { (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf)); Configuration conf2 = tableSource2.getConf(); assertNull(conf2.get(FlinkOptions.SOURCE_AVRO_SCHEMA), "expect schema string as null"); + + // infer special avro data types that needs namespace + this.conf.removeConfig(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH); + ResolvedSchema schema3 = SchemaBuilder.instance() + .field("f_decimal", DataTypes.DECIMAL(3, 2).notNull()) + .field("f_map", DataTypes.MAP(DataTypes.VARCHAR(20), DataTypes.VARCHAR(10))) + .field("f_array", DataTypes.ARRAY(DataTypes.VARCHAR(10))) + .field("f_record", DataTypes.ROW(DataTypes.FIELD("r1", DataTypes.VARCHAR(10)), DataTypes.FIELD("r2", DataTypes.INT()))) + .primaryKey("f_decimal") + .build(); + final HoodieTableSink tableSink3 = + (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf, schema3, "")); + final Configuration conf3 = tableSink3.getConf(); + final String expected = AvroSchemaConverter.convertToSchema(schema3.toSourceRowDataType().getLogicalType(), AvroSchemaUtils.getAvroRecordQualifiedName("t1")).toString(); + assertThat(conf3.get(FlinkOptions.SOURCE_AVRO_SCHEMA), is(expected)); } @Test @@ -367,6 +463,21 @@ void testInferAvroSchemaForSink() { (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf)); Configuration conf2 = tableSink2.getConf(); assertNull(conf2.get(FlinkOptions.SOURCE_AVRO_SCHEMA), "expect schema string as null"); + + // infer special avro data types that needs namespace + this.conf.removeConfig(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH); + ResolvedSchema schema3 = SchemaBuilder.instance() + .field("f_decimal", DataTypes.DECIMAL(3, 2).notNull()) + .field("f_map", DataTypes.MAP(DataTypes.VARCHAR(20), DataTypes.VARCHAR(10))) + .field("f_array", DataTypes.ARRAY(DataTypes.VARCHAR(10))) + .field("f_record", DataTypes.ROW(DataTypes.FIELD("r1", DataTypes.VARCHAR(10)), DataTypes.FIELD("r2", DataTypes.INT()))) + .primaryKey("f_decimal") + .build(); + final HoodieTableSink tableSink3 = + (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf, schema3, "")); + final Configuration conf3 = tableSink3.getConf(); + final String expected = AvroSchemaConverter.convertToSchema(schema3.toSinkRowDataType().getLogicalType(), AvroSchemaUtils.getAvroRecordQualifiedName("t1")).toString(); + assertThat(conf3.get(FlinkOptions.SOURCE_AVRO_SCHEMA), is(expected)); } @Test diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java index 1246e140a5fb6..5983192fc8221 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 @@ -18,12 +18,16 @@ package org.apache.hudi.table.catalog; +import org.apache.hudi.common.model.DefaultHoodieRecordPayload; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; +import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.sink.partitioner.profile.WriteProfiles; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; @@ -54,6 +58,7 @@ import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; 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; @@ -109,7 +114,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; } @@ -150,14 +159,19 @@ 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<>(); assertThrows(ValidationException.class, () -> catalog = new HoodieCatalog("hudi", Configuration.fromMap(catalogOptions))); catalogPathStr = tempFile.getAbsolutePath(); catalogOptions.put(CATALOG_PATH.key(), catalogPathStr); catalogOptions.put(DEFAULT_DATABASE.key(), TEST_DEFAULT_DATABASE); - catalog = new HoodieCatalog("hudi", Configuration.fromMap(catalogOptions)); - catalog.open(); + return catalogOptions; } @AfterEach @@ -222,11 +236,41 @@ public void testCreateTable() throws Exception { // test table exist assertTrue(catalog.tableExists(tablePath)); + // validate the full name of table create schema + HoodieTableConfig tableConfig = StreamerUtil.getTableConfig( + catalog.getTable(tablePath).getOptions().get(FlinkOptions.PATH.key()), + HadoopConfigurations.getHadoopConf(new Configuration())).get(); + Option tableCreateSchema = tableConfig.getTableCreateSchema(); + assertTrue(tableCreateSchema.isPresent(), "Table should have been created"); + assertThat(tableCreateSchema.get().getFullName(), is("hoodie.tb1.tb1_record")); + // test create exist table assertThrows(TableAlreadyExistException.class, () -> 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 5d27cdadbbb35..7695a205e2b67 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 @@ -22,8 +22,10 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieCatalogException; import org.apache.hudi.sink.partitioner.profile.WriteProfiles; @@ -54,6 +56,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; @@ -139,6 +142,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())); @@ -153,6 +171,12 @@ public void testCreateAndGetHoodieTable(HoodieTableType tableType) throws Except assertEquals(Collections.singletonList("uuid"), table1.getUnresolvedSchema().getPrimaryKey().get().getColumnNames()); assertEquals(Collections.singletonList("par1"), ((CatalogTable) table1).getPartitionKeys()); + // validate the full name of table create schema + HoodieTableConfig tableConfig = StreamerUtil.getTableConfig(table1.getOptions().get(FlinkOptions.PATH.key()), hoodieCatalog.getHiveConf()).get(); + Option tableCreateSchema = tableConfig.getTableCreateSchema(); + assertTrue(tableCreateSchema.isPresent(), "Table should have been created"); + assertThat(tableCreateSchema.get().getFullName(), is("hoodie.test.test_record")); + // validate explicit primary key options.put(FlinkOptions.RECORD_KEY_FIELD.key(), "id"); table = new CatalogTableImpl(schema, partitions, options, "hudi table"); @@ -192,6 +216,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 6d0bf731ccc35..7563498bbb61b 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 @@ -324,7 +324,7 @@ void testReadWithDeletesCOW() throws Exception { void testReadWithChangeLogCOW(HoodieCDCSupplementalLoggingMode mode) throws Exception { Map options = new HashMap<>(); options.put(FlinkOptions.CDC_ENABLED.key(), "true"); - options.put(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE.key(), mode.getValue()); + options.put(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE.key(), mode.name()); beforeEach(HoodieTableType.COPY_ON_WRITE, options); // write the insert data sets @@ -365,7 +365,7 @@ void testReadWithChangeLogCOW(HoodieCDCSupplementalLoggingMode mode) throws Exce void testReadFromEarliestWithChangeLogCOW(HoodieCDCSupplementalLoggingMode mode) throws Exception { Map options = new HashMap<>(); options.put(FlinkOptions.CDC_ENABLED.key(), "true"); - options.put(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE.key(), mode.getValue()); + options.put(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE.key(), mode.name()); options.put(FlinkOptions.READ_START_COMMIT.key(), "earliest"); beforeEach(HoodieTableType.COPY_ON_WRITE, options); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/util/TestExpressionUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/util/TestExpressionUtils.java new file mode 100644 index 0000000000000..50816a298de0a --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/util/TestExpressionUtils.java @@ -0,0 +1,151 @@ +/* + * 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.table.api.DataTypes; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoField; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TestExpressionUtils { + + private static final DataType ROW_DATA_TYPE = DataTypes.ROW( + DataTypes.FIELD("f_tinyint", DataTypes.TINYINT()), + DataTypes.FIELD("f_smallint", DataTypes.SMALLINT()), + DataTypes.FIELD("f_int", DataTypes.INT()), + DataTypes.FIELD("f_long", DataTypes.BIGINT()), + DataTypes.FIELD("f_float", DataTypes.FLOAT()), + DataTypes.FIELD("f_double", DataTypes.DOUBLE()), + DataTypes.FIELD("f_boolean", DataTypes.BOOLEAN()), + DataTypes.FIELD("f_decimal", DataTypes.DECIMAL(10, 2)), + DataTypes.FIELD("f_bytes", DataTypes.VARBINARY(10)), + DataTypes.FIELD("f_string", DataTypes.VARCHAR(10)), + DataTypes.FIELD("f_time", DataTypes.TIME(3)), + DataTypes.FIELD("f_date", DataTypes.DATE()), + DataTypes.FIELD("f_timestamp", DataTypes.TIMESTAMP(3)) + ).notNull(); + + private static final DataType ROW_DATA_TYPE_FIELD_NON_NULL = DataTypes.ROW( + DataTypes.FIELD("f_tinyint", DataTypes.TINYINT().notNull()), + DataTypes.FIELD("f_smallint", DataTypes.SMALLINT().notNull()), + DataTypes.FIELD("f_int", DataTypes.INT().notNull()), + DataTypes.FIELD("f_long", DataTypes.BIGINT().notNull()), + DataTypes.FIELD("f_float", DataTypes.FLOAT().notNull()), + DataTypes.FIELD("f_double", DataTypes.DOUBLE().notNull()), + DataTypes.FIELD("f_boolean", DataTypes.BOOLEAN().notNull()), + DataTypes.FIELD("f_decimal", DataTypes.DECIMAL(10, 2).notNull()), + DataTypes.FIELD("f_bytes", DataTypes.VARBINARY(10).notNull()), + DataTypes.FIELD("f_string", DataTypes.VARCHAR(10).notNull()), + DataTypes.FIELD("f_time", DataTypes.TIME(3).notNull()), + DataTypes.FIELD("f_date", DataTypes.DATE().notNull()), + DataTypes.FIELD("f_timestamp", DataTypes.TIMESTAMP(3).notNull()) + ).notNull(); + + @Test + void getValueFromLiteralForNull() { + List fields = ((RowType) ROW_DATA_TYPE.getLogicalType()).getFields(); + List dataTypes = ROW_DATA_TYPE.getChildren(); + CallExpression callExpression; + for (int i = 0; i < fields.size(); i++) { + // 1. Build all types + callExpression = new CallExpression( + BuiltInFunctionDefinitions.IS_NOT_NULL, + Arrays.asList(new FieldReferenceExpression(fields.get(i).getName(), + dataTypes.get(i), + 2, + 2), new ValueLiteralExpression(null, dataTypes.get(i))), + DataTypes.BOOLEAN()); + List childExprs = callExpression.getChildren(); + + // 2. Parse each type + boolean hasNullLiteral = + childExprs.stream().anyMatch(e -> + e instanceof ValueLiteralExpression + && ExpressionUtils.getValueFromLiteral((ValueLiteralExpression) e) == null); + assertTrue(hasNullLiteral); + } + + } + + @Test + void getValueFromLiteralForNonNull() { + List fields = ((RowType) ROW_DATA_TYPE_FIELD_NON_NULL.getLogicalType()).getFields(); + List dataTypes = ROW_DATA_TYPE_FIELD_NON_NULL.getChildren(); + // tests for non-null literals + List dataList = new ArrayList<>(fields.size()); + dataList.add(new Byte("1")); // f_tinyint + dataList.add(new Short("2")); // f_smallint + dataList.add(new Integer("3")); // f_int + dataList.add(new Long("4")); // f_long + dataList.add(new Float(5.0)); // f_float + dataList.add(new Double(6.0)); // f_double + dataList.add(new Boolean(true)); // f_boolean + dataList.add(new BigDecimal(3.0)); // f_decimal + dataList.add("hudi".getBytes(StandardCharsets.UTF_8)); // f_bytes + dataList.add("hudi ok"); // f_string + dataList.add(LocalTime.of(1, 11, 11)); // f_time + dataList.add(LocalDate.of(2023, 1, 2)); // f_date + dataList.add(LocalDateTime.of(2023, 1, 2, 3, 4)); // f_timestamp + CallExpression callExpression; + for (int i = 0; i < fields.size(); i++) { + // 1. Build all types + callExpression = new CallExpression( + BuiltInFunctionDefinitions.IS_NOT_NULL, + Arrays.asList( + new FieldReferenceExpression( + fields.get(i).getName(), + dataTypes.get(i), + i, + i), + new ValueLiteralExpression(dataList.get(i), dataTypes.get(i))), + DataTypes.BOOLEAN()); + List childExprs = callExpression.getChildren(); + // 2. Parse each type + if (dataList.get(i) instanceof LocalTime) { + assertEquals(((LocalTime) dataList.get(i)).get(ChronoField.MILLI_OF_DAY), ExpressionUtils.getValueFromLiteral((ValueLiteralExpression) childExprs.get(1))); + } else if (dataList.get(i) instanceof LocalDate) { + assertEquals(((LocalDate) dataList.get(i)).toEpochDay(), ExpressionUtils.getValueFromLiteral((ValueLiteralExpression) childExprs.get(1))); + } else if (dataList.get(i) instanceof LocalDateTime) { + assertEquals(((LocalDateTime) dataList.get(i)).toInstant(ZoneOffset.UTC).toEpochMilli(), ExpressionUtils.getValueFromLiteral((ValueLiteralExpression) childExprs.get(1))); + } else { + assertEquals(dataList.get(i), ExpressionUtils.getValueFromLiteral((ValueLiteralExpression) childExprs.get(1))); + } + } + } +} \ No newline at end of file diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java index a641811bb738a..d3bdc479d318b 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestStreamerUtil.java @@ -18,13 +18,18 @@ package org.apache.hudi.utils; +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.util.FileIOUtils; import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.util.StreamerUtil; import org.apache.flink.configuration.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -101,5 +106,21 @@ void testInstantTimeDiff() { long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower); assertThat(diff, is(75L)); } + + @Test + void testTableExist() throws IOException { + Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()); + String basePath = tempFile.getAbsolutePath(); + + assertFalse(StreamerUtil.tableExists(basePath, HadoopConfigurations.getHadoopConf(conf))); + + try (FileSystem fs = FSUtils.getFs(basePath, HadoopConfigurations.getHadoopConf(conf))) { + fs.mkdirs(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)); + assertFalse(StreamerUtil.tableExists(basePath, HadoopConfigurations.getHadoopConf(conf))); + + fs.create(new Path(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME), HoodieTableConfig.HOODIE_PROPERTIES_FILE)); + assertTrue(StreamerUtil.tableExists(basePath, HadoopConfigurations.getHadoopConf(conf))); + } + } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java index bd437a9070dcd..46af5fa7a82e0 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java @@ -107,4 +107,14 @@ public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) { final String basePath = conf.getString(FlinkOptions.PATH); return new StreamReadMonitoringFunction(conf, new Path(basePath), TestConfigurations.ROW_TYPE, 1024 * 1024L, null); } + + public static int getCompletedInstantCount(String basePath, String action) { + final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() + .setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build(); + return metaClient.getActiveTimeline() + .filterCompletedInstants() + .filter(instant -> action.equals(instant.getAction())) + .countInstants(); + } + } diff --git a/hudi-flink-datasource/hudi-flink/src/test/resources/test_read_schema_dropped_age.avsc b/hudi-flink-datasource/hudi-flink/src/test/resources/test_read_schema_dropped_age.avsc new file mode 100644 index 0000000000000..9bb99016cae94 --- /dev/null +++ b/hudi-flink-datasource/hudi-flink/src/test/resources/test_read_schema_dropped_age.avsc @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "type" : "record", + "name" : "record", + "fields" : [ { + "name" : "uuid", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "name", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "ts", + "type" : [ "null", { + "type" : "long", + "logicalType" : "timestamp-millis" + } ], + "default" : null + }, { + "name" : "partition", + "type" : [ "null", "string" ], + "default" : null + } ] +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml index 11c6a9673c5dd..98fc4e5bbbbdd 100644 --- a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink1.13.x - 0.13.0-SNAPSHOT + 0.13.1 jar diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java new file mode 100644 index 0000000000000..ea0ba0419214b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** Adapter class for {@code Output} to handle async compaction/clustering service thread safe issues */ +public class MaskingOutputAdapter implements Output> { + + private final Output> output; + + public MaskingOutputAdapter(Output> output) { + this.output = output; + } + + @Override + public void emitWatermark(Watermark watermark) { + // For thread safe, not to propagate the watermark + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + // For thread safe, not to propagate latency marker + } + + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + this.output.collect(outputTag, streamRecord); + } + + @Override + public void collect(StreamRecord outStreamRecord) { + this.output.collect(outStreamRecord); + } + + @Override + public void close() { + this.output.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java index 555853bda6bd8..70638a9c43200 100644 --- a/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java +++ b/hudi-flink-datasource/hudi-flink1.13.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -59,7 +59,7 @@ public Int64TimestampColumnReader( throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } diff --git a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml index d1c7a157e91cc..26b75b90f4157 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink1.14.x - 0.13.0-SNAPSHOT + 0.13.1 jar diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java new file mode 100644 index 0000000000000..e84da0d6ec30b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.util.OutputTag; + +/** Adapter class for {@code Output} to handle async compaction/clustering service thread safe issues */ +public class MaskingOutputAdapter implements Output> { + + private final Output> output; + + public MaskingOutputAdapter(Output> output) { + this.output = output; + } + + @Override + public void emitWatermark(Watermark watermark) { + // For thread safe, not to propagate the watermark + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + // For thread safe, not to propagate latency marker + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // For thread safe, not to propagate watermark status + } + + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + this.output.collect(outputTag, streamRecord); + } + + @Override + public void collect(StreamRecord outStreamRecord) { + this.output.collect(outStreamRecord); + } + + @Override + public void close() { + this.output.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java index 555853bda6bd8..70638a9c43200 100644 --- a/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java +++ b/hudi-flink-datasource/hudi-flink1.14.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -59,7 +59,7 @@ public Int64TimestampColumnReader( throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } diff --git a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml index 8262407b96f83..6f3860c247c45 100644 --- a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink1.15.x - 0.13.0-SNAPSHOT + 0.13.1 jar diff --git a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java new file mode 100644 index 0000000000000..e84da0d6ec30b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.util.OutputTag; + +/** Adapter class for {@code Output} to handle async compaction/clustering service thread safe issues */ +public class MaskingOutputAdapter implements Output> { + + private final Output> output; + + public MaskingOutputAdapter(Output> output) { + this.output = output; + } + + @Override + public void emitWatermark(Watermark watermark) { + // For thread safe, not to propagate the watermark + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + // For thread safe, not to propagate latency marker + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // For thread safe, not to propagate watermark status + } + + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + this.output.collect(outputTag, streamRecord); + } + + @Override + public void collect(StreamRecord outStreamRecord) { + this.output.collect(outStreamRecord); + } + + @Override + public void close() { + this.output.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java index 417b1155bbd7b..b44273b57ca26 100644 --- a/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java +++ b/hudi-flink-datasource/hudi-flink1.15.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -59,7 +59,7 @@ public Int64TimestampColumnReader( throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } diff --git a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml index a9966e8255a47..8dfc65b8c932c 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml +++ b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml @@ -20,12 +20,12 @@ hudi-flink-datasource org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink1.16.x - 0.13.0-SNAPSHOT + 0.13.1 jar diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java new file mode 100644 index 0000000000000..e84da0d6ec30b --- /dev/null +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/adapter/MaskingOutputAdapter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.adapter; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.util.OutputTag; + +/** Adapter class for {@code Output} to handle async compaction/clustering service thread safe issues */ +public class MaskingOutputAdapter implements Output> { + + private final Output> output; + + public MaskingOutputAdapter(Output> output) { + this.output = output; + } + + @Override + public void emitWatermark(Watermark watermark) { + // For thread safe, not to propagate the watermark + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + // For thread safe, not to propagate latency marker + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) { + // For thread safe, not to propagate watermark status + } + + @Override + public void collect(OutputTag outputTag, StreamRecord streamRecord) { + this.output.collect(outputTag, streamRecord); + } + + @Override + public void collect(StreamRecord outStreamRecord) { + this.output.collect(outStreamRecord); + } + + @Override + public void close() { + this.output.close(); + } +} diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java index 417b1155bbd7b..b44273b57ca26 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java +++ b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -59,7 +59,7 @@ public Int64TimestampColumnReader( throw new IllegalArgumentException( "Avro does not support TIMESTAMP type with precision: " + precision - + ", it only supports precision less than 6."); + + ", it only support precisions <= 6."); } checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } diff --git a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java b/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index cd1bb59c34d40..1872ec385b4a9 100644 --- a/hudi-flink-datasource/hudi-flink1.16.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java +++ b/hudi-flink-datasource/hudi-flink1.16.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-flink-datasource/pom.xml b/hudi-flink-datasource/pom.xml index b776d8e9db829..42e3b465424f1 100644 --- a/hudi-flink-datasource/pom.xml +++ b/hudi-flink-datasource/pom.xml @@ -20,12 +20,12 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-flink-datasource - 0.13.0-SNAPSHOT + 0.13.1 pom diff --git a/hudi-gcp/pom.xml b/hudi-gcp/pom.xml index c70e0af1ed7e5..d27dc5e460a29 100644 --- a/hudi-gcp/pom.xml +++ b/hudi-gcp/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../pom.xml 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..8c04c0fa13652 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 @@ -19,20 +19,27 @@ package org.apache.hudi.gcp.bigquery; +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; 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 javax.annotation.concurrent.Immutable; + import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; import java.util.Properties; /** * Configs needed to sync data into BigQuery. */ +@Immutable +@ConfigClassProperty(name = "BigQuery Sync Configs", + groupName = ConfigGroups.Names.META_SYNC, + description = "Configurations used by the Hudi to sync metadata to Google BigQuery.") public class BigQuerySyncConfig extends HoodieSyncConfig implements Serializable { public static final ConfigProperty BIGQUERY_SYNC_PROJECT_ID = ConfigProperty @@ -101,38 +108,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/pom.xml b/hudi-hadoop-mr/pom.xml index a358e720a2e18..51e17711737f3 100644 --- a/hudi-hadoop-mr/pom.xml +++ b/hudi-hadoop-mr/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index c168924e65fe0..d9e06deb2cabe 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -18,6 +18,11 @@ package org.apache.hudi.hadoop; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.hadoop.utils.HoodieHiveUtils; + +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg; import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.io.ArrayWritable; @@ -27,9 +32,8 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.hadoop.utils.HoodieHiveUtils; +import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -83,6 +87,7 @@ public RecordReader getRecordReader(final InputSpli LOG.debug("EMPLOYING DEFAULT RECORD READER - " + split); } + HoodieRealtimeInputFormatUtils.addProjectionField(job, job.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "").split("/")); return getRecordReaderInternal(split, job, reporter); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java index cdc062475ffcf..b413164a45cba 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieHFileRealtimeInputFormat.java @@ -72,7 +72,7 @@ public RecordReader getRecordReader(final InputSpli // For e:g _hoodie_record_key would be missing and merge step would throw exceptions. // TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction // time. - HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf, Option.empty(), Option.empty()); + HoodieRealtimeInputFormatUtils.addVirtualKeysProjection(jobConf, Option.empty()); this.conf = jobConf; this.conf.set(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP, "true"); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index 78768104d9974..60ee704551f11 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -40,6 +40,9 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; /** * Input Format, that provides a real-time view of data in a Hoodie table. @@ -68,7 +71,7 @@ public RecordReader getRecordReader(final InputSpli // add preCombineKey HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jobConf).setBasePath(realtimeSplit.getBasePath()).build(); HoodieTableConfig tableConfig = metaClient.getTableConfig(); - addProjectionToJobConf(realtimeSplit, jobConf, metaClient.getTableConfig().getPreCombineField()); + addProjectionToJobConf(realtimeSplit, jobConf, tableConfig); LOG.info("Creating record reader with readCols :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR) + ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); @@ -81,7 +84,7 @@ public RecordReader getRecordReader(final InputSpli super.getRecordReader(split, jobConf, reporter)); } - void addProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf jobConf, String preCombineKey) { + void addProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf jobConf, HoodieTableConfig tableConfig) { // Hive on Spark invokes multiple getRecordReaders from different threads in the same spark task (and hence the // same JVM) unlike Hive on MR. Due to this, accesses to JobConf, which is shared across all threads, is at the // risk of experiencing race conditions. Hence, we synchronize on the JobConf object here. There is negligible @@ -100,10 +103,21 @@ void addProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf job // For e:g _hoodie_record_key would be missing and merge step would throw exceptions. // TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction // time. + List fieldsToAdd = new ArrayList<>(); if (!realtimeSplit.getDeltaLogPaths().isEmpty()) { - HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf, realtimeSplit.getVirtualKeyInfo(), - StringUtils.isNullOrEmpty(preCombineKey) ? Option.empty() : Option.of(preCombineKey)); + HoodieRealtimeInputFormatUtils.addVirtualKeysProjection(jobConf, realtimeSplit.getVirtualKeyInfo()); + String preCombineKey = tableConfig.getPreCombineField(); + if (!StringUtils.isNullOrEmpty(preCombineKey)) { + fieldsToAdd.add(preCombineKey); + } } + + Option partitions = tableConfig.getPartitionFields(); + if (partitions.isPresent()) { + fieldsToAdd.addAll(Arrays.asList(partitions.get())); + } + HoodieRealtimeInputFormatUtils.addProjectionField(jobConf, fieldsToAdd.toArray(new String[0])); + jobConf.set(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP, "true"); setConf(jobConf); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index 569230634781c..cf16bf0bd80ed 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -100,7 +100,7 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept .withDiskMapType(jobConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())) .withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())) - .withUseScanV2(jobConf.getBoolean(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, false)) + .withOptimizedLogBlocksScan(jobConf.getBoolean(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, false)) .withInternalSchema(schemaEvolutionContext.internalSchemaOption.orElse(InternalSchema.getEmptyInternalSchema())) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 44e6dd7f932de..bf83f43cb4232 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -70,7 +70,7 @@ private static Configuration addProjectionField(Configuration conf, String field readColIdsPrefix = ""; } - if (!readColNames.contains(fieldName)) { + if (!Arrays.asList(readColNames.split(",")).contains(fieldName)) { // If not already in the list - then add it conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, readColNamesPrefix + fieldName); conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, readColIdsPrefix + fieldIndex); @@ -83,7 +83,19 @@ private static Configuration addProjectionField(Configuration conf, String field return conf; } - public static void addRequiredProjectionFields(Configuration configuration, Option hoodieVirtualKeyInfo, Option preCombineKeyOpt) { + public static void addProjectionField(Configuration conf, String[] fieldName) { + if (fieldName.length > 0) { + List columnNameList = Arrays.stream(conf.get(serdeConstants.LIST_COLUMNS).split(",")).collect(Collectors.toList()); + Arrays.stream(fieldName).forEach(field -> { + int index = columnNameList.indexOf(field); + if (index != -1) { + addProjectionField(conf, field, index); + } + }); + } + } + + public static void addVirtualKeysProjection(Configuration configuration, Option hoodieVirtualKeyInfo) { // Need this to do merge records in HoodieRealtimeRecordReader if (!hoodieVirtualKeyInfo.isPresent()) { addProjectionField(configuration, HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS); @@ -96,18 +108,6 @@ public static void addRequiredProjectionFields(Configuration configuration, Opti addProjectionField(configuration, hoodieVirtualKey.getPartitionPathField().get(), hoodieVirtualKey.getPartitionPathFieldIndex().get()); } } - - if (preCombineKeyOpt.isPresent()) { - // infer col pos - String preCombineKey = preCombineKeyOpt.get(); - List columnNameList = Arrays.stream(configuration.get(serdeConstants.LIST_COLUMNS).split(",")).collect(Collectors.toList()); - int pos = columnNameList.indexOf(preCombineKey); - if (pos != -1) { - addProjectionField(configuration, preCombineKey, pos); - LOG.info(String.format("add preCombineKey: %s to project columns with position %s", preCombineKey, pos)); - } - } - } public static boolean requiredProjectionFieldsExistInConf(Configuration configuration, Option hoodieVirtualKeyInfo) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java index da392492294c2..9e3a92a156270 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java @@ -257,10 +257,8 @@ public static List orderFields(String fieldNameCsv, String fieldOrderCsv String[] fieldOrdersWithDups = fieldOrderCsv.isEmpty() ? new String[0] : fieldOrderCsv.split(","); Set fieldOrdersSet = new LinkedHashSet<>(Arrays.asList(fieldOrdersWithDups)); String[] fieldOrders = fieldOrdersSet.toArray(new String[0]); - List fieldNames = fieldNameCsv.isEmpty() ? new ArrayList<>() : Arrays.stream(fieldNameCsv.split(",")) - .filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList()); + List fieldNames = fieldNameCsv.isEmpty() ? new ArrayList<>() : Arrays.stream(fieldNameCsv.split(",")).collect(Collectors.toList()); Set fieldNamesSet = new LinkedHashSet<>(fieldNames); - // Hive does not provide ids for partitioning fields, so check for lengths excluding that. if (fieldNamesSet.size() != fieldOrders.length) { throw new HoodieException(String .format("Error ordering fields for storage read. #fieldNames: %d, #fieldPositions: %d", diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml index 911daada76767..910d40bd1784c 100644 --- a/hudi-integ-test/pom.xml +++ b/hudi-integ-test/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../pom.xml hudi-integ-test @@ -51,13 +51,18 @@ test - - - org.eclipse.jetty.aggregate - jetty-all - ${jetty.version} - uber + com.kohlschutter.junixsocket + junixsocket-common + 2.6.2 + jar + test + + + com.kohlschutter.junixsocket + junixsocket-native-common + 2.6.2 + jar test @@ -564,4 +569,19 @@ + + + + m1-mac + + ${project.basedir}/../docker/compose/docker-compose_hadoop284_hive233_spark244_mac_aarch64.yml + + + + mac + aarch64 + + + + 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-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index 3ed86c32c948c..0e3f27e2e3b10 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -290,7 +290,7 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue()) .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()) .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()) - .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue())) + .withOptimizedLogBlocksScan(Boolean.parseBoolean(HoodieCompactionConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN.defaultValue())) .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) .build(); // readAvro log files diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeleteNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeleteNode.scala index ecf94b94ec975..ac6c93d15fb3a 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeleteNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeleteNode.scala @@ -64,7 +64,7 @@ class SparkDeleteNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { context.getWriterContext.getSparkSession) inputDF.write.format("hudi") - .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .options(DataSourceWriteOptions.mayBeDerivePartitionPath(context.getWriterContext.getProps.asScala.toMap)) .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala index 9354deea28bb0..1abac27bb6140 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkDeletePartitionNode.scala @@ -57,7 +57,7 @@ class SparkDeletePartitionNode(dagNodeConfig: Config) extends DagNode[RDD[WriteS println("Generating input data for node {}", this.getName) context.getWriterContext.getSparkSession.emptyDataFrame.write.format("hudi") - .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .options(DataSourceWriteOptions.mayBeDerivePartitionPath(context.getWriterContext.getProps.asScala.toMap)) .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), SchemaUtils.SOURCE_ORDERING_FIELD) .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala index 6c1d39e2f6c37..9cc1b0160d80e 100644 --- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala +++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala @@ -66,7 +66,7 @@ class SparkInsertNode(dagNodeConfig: Config) extends DagNode[RDD[WriteStatus]] { context.getWriterContext.getSparkSession) inputDF.write.format("hudi") - .options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap)) + .options(DataSourceWriteOptions.mayBeDerivePartitionPath(context.getWriterContext.getProps.asScala.toMap)) .option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "test_suite_source_ordering_field") .option(DataSourceWriteOptions.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName) .option(DataSourceWriteOptions.TABLE_TYPE.key, context.getHoodieTestSuiteWriter.getCfg.tableType) diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java index a050d7eb88ba8..562c69b722119 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieSanity.java @@ -27,6 +27,9 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.util.Arrays; +import java.util.stream.Collectors; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -192,6 +195,17 @@ public void testRunHoodieJavaApp(String command, String hiveTableName, String ta stdOutErr = executeHiveCommand("select count(1) from " + roTableName.get()); assertEquals(80, Integer.parseInt(lastLine(stdOutErr.getLeft()).trim()), "Expecting 80 rows to be present in the snapshot table"); + + if (partitionType != PartitionType.NON_PARTITIONED) { + // Verify queries with partition field predicates, some partitions may be empty, so we query all the partitions. + String[] partitions = getPartitions(roTableName.get()); + assertTrue(partitions.length > 0); + String partitionClause = partitionType == PartitionType.SINGLE_KEY_PARTITIONED + ? Arrays.stream(partitions).map(String::trim).collect(Collectors.joining(" or ")) + : Arrays.stream(partitions).map(par -> String.join(" and ", par.trim().split("/"))).collect(Collectors.joining(" or ")); + stdOutErr = executeHiveCommand("select * from " + roTableName.get() + " where " + partitionClause); + assertTrue(stdOutErr.getLeft().split("\n").length > 0, "Expecting at least one row to be present, but got " + stdOutErr); + } } // Make the HDFS dataset non-hoodie and run the same query; Checks for interoperability with non-hoodie tables @@ -209,6 +223,11 @@ public void testRunHoodieJavaApp(String command, String hiveTableName, String ta "Expecting 280 rows to be present in the new table"); } + private String[] getPartitions(String tableName) throws Exception { + Pair stdOutErr = executeHiveCommand("show partitions " + tableName); + return stdOutErr.getLeft().split("\n"); + } + private static String lastLine(String output) { String[] lines = output.split("\n"); return lines[lines.length - 1]; diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml index 8b233fb21e748..04b3f525fbaca 100644 --- a/hudi-kafka-connect/pom.xml +++ b/hudi-kafka-connect/pom.xml @@ -19,13 +19,13 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 hudi-kafka-connect Kafka Connect Sink Connector for Hudi - 0.13.0-SNAPSHOT + 0.13.1 jar diff --git a/hudi-platform-service/hudi-metaserver/README.md b/hudi-platform-service/hudi-metaserver/README.md index a7802e5589122..a1d59597491ce 100644 --- a/hudi-platform-service/hudi-metaserver/README.md +++ b/hudi-platform-service/hudi-metaserver/README.md @@ -39,10 +39,10 @@ Attention: Apple m1 cannot install thrift by docker successfully. The script wil ### Source code generated by Thrift -After packaging, the generated source code are placed in `target/generated-sources/thrift/gen-java`. +After packaging, the generated source code are placed in `target/generated-sources/gen-java`. It looks like, -```shell +```text ├── gen-java │ └── org │ └── apache @@ -59,7 +59,7 @@ It looks like, ### Start Hudi Metaserver 1. modify the `hikariPool.properties` and config the mysql address. For example, -```text +```properties jdbcUrl=jdbc:mysql://localhost:3306 dataSource.user=root dataSource.password=password @@ -73,7 +73,7 @@ sh start_hudi_metaserver.sh ### Write client configurations -```shell +```properties hoodie.database.name=default hoodie.table.name=test hoodie.base.path=${path} diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml index d8fb242f80bd8..7fb901153caaf 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml @@ -21,7 +21,7 @@ hudi-metaserver org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 @@ -32,11 +32,6 @@ - - org.apache.hudi - hudi-common - ${project.version} - org.apache.hudi hudi-client-common @@ -44,14 +39,6 @@ - - org.apache.hudi - hudi-common - ${project.version} - tests - test-jar - test - org.apache.hudi hudi-client-common @@ -88,4 +75,4 @@ - \ No newline at end of file + diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml index 6a008717effbd..c62f4a3406ef3 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml @@ -21,7 +21,7 @@ hudi-metaserver org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 4.0.0 @@ -147,4 +147,4 @@ - \ No newline at end of file + diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/resources/mybatis-config.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/resources/mybatis-config.xml index 8610f8983ddac..6259fc196cfe1 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/resources/mybatis-config.xml +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/resources/mybatis-config.xml @@ -1,12 +1,13 @@ + +# Bundle Validation for Hudi + +This directory contains scripts for running bundle validation in Github Actions (`validate-bundles` +specified in `.github/workflows/bot.yml`) and build profile for Docker images used. + +## Docker Image for Bundle Validation + +The base image for bundle validation is pre-built and upload to the Docker Hub: +https://hub.docker.com/r/apachehudi/hudi-ci-bundle-validation-base. + +The `Dockerfile` for the image is under `base/`. To build the image with updated `Dockerfile`, you may use the script in +the folder. Here are the docker commands to build the image by specifying different versions: + +```shell +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.13.6 \ + --build-arg SPARK_VERSION=3.1.3 \ + --build-arg SPARK_HADOOP_VERSION=2.7 \ + -t hudi-ci-bundle-validation-base:flink1136hive313spark313 . +docker image tag hudi-ci-bundle-validation-base:flink1136hive313spark313 apachehudi/hudi-ci-bundle-validation-base:flink1136hive313spark313 +``` + +To upload the image with the tag: + +```shell +docker push apachehudi/hudi-ci-bundle-validation-base:flink1136hive313spark313 +``` + +Note that for each library like Hive and Spark, the download and extraction happen under one `RUN` instruction so that +only one layer is generated to limit the size of the image. However, this makes repeated downloads when rebuilding the +image. If you need faster iteration for local build, you may use the `Dockerfile` under `base-dev/`, which uses `ADD` +instruction for downloads, which provides caching across builds. This increases the size of the generated image compared +to `base/` and the image should only be used for development only and not be pushed to remote. \ No newline at end of file diff --git a/packaging/bundle-validation/base-dev/Dockerfile b/packaging/bundle-validation/base-dev/Dockerfile new file mode 100644 index 0000000000000..a1f3ba88e646e --- /dev/null +++ b/packaging/bundle-validation/base-dev/Dockerfile @@ -0,0 +1,77 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +FROM adoptopenjdk/openjdk8:alpine + +RUN apk add --no-cache --upgrade bash curl jq openjdk11 openjdk17 --repository=https://dl-cdn.alpinelinux.org/alpine/v3.15/community + +RUN mkdir /opt/bundle-validation +ENV WORKDIR=/opt/bundle-validation +WORKDIR $WORKDIR + +ARG HADOOP_VERSION=2.7.7 +ARG HIVE_VERSION=3.1.3 +ARG DERBY_VERSION=10.14.1.0 +ARG FLINK_VERSION=1.13.6 +ARG SPARK_VERSION=3.1.3 +ARG SPARK_HADOOP_VERSION=2.7 +ARG CONFLUENT_MAJOR_VERSION=5.5 +ARG CONFLUENT_VERSION=5.5.12 +ARG KAFKA_CONNECT_HDFS_VERSION=10.1.13 + +ADD https://archive.apache.org/dist/hadoop/common/hadoop-$HADOOP_VERSION/hadoop-$HADOOP_VERSION.tar.gz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/hadoop-$HADOOP_VERSION.tar.gz -C $WORKDIR/ \ + && rm $WORKDIR/hadoop-$HADOOP_VERSION.tar.gz +ENV HADOOP_HOME=$WORKDIR/hadoop-$HADOOP_VERSION + +ADD https://archive.apache.org/dist/hive/hive-$HIVE_VERSION/apache-hive-$HIVE_VERSION-bin.tar.gz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/apache-hive-$HIVE_VERSION-bin.tar.gz -C $WORKDIR/ \ + && rm $WORKDIR/apache-hive-$HIVE_VERSION-bin.tar.gz +ENV HIVE_HOME=$WORKDIR/apache-hive-$HIVE_VERSION-bin + +ADD https://archive.apache.org/dist/db/derby/db-derby-$DERBY_VERSION/db-derby-$DERBY_VERSION-bin.tar.gz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/db-derby-$DERBY_VERSION-bin.tar.gz -C $WORKDIR/ \ + && rm $WORKDIR/db-derby-$DERBY_VERSION-bin.tar.gz +ENV DERBY_HOME=$WORKDIR/db-derby-$DERBY_VERSION-bin + +ADD https://archive.apache.org/dist/flink/flink-$FLINK_VERSION/flink-$FLINK_VERSION-bin-scala_2.12.tgz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/flink-$FLINK_VERSION-bin-scala_2.12.tgz -C $WORKDIR/ \ + && rm $WORKDIR/flink-$FLINK_VERSION-bin-scala_2.12.tgz +ENV FLINK_HOME=$WORKDIR/flink-$FLINK_VERSION + +ADD https://archive.apache.org/dist/spark/spark-$SPARK_VERSION/spark-$SPARK_VERSION-bin-hadoop$SPARK_HADOOP_VERSION.tgz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/spark-$SPARK_VERSION-bin-hadoop$SPARK_HADOOP_VERSION.tgz -C $WORKDIR/ \ + && rm $WORKDIR/spark-$SPARK_VERSION-bin-hadoop$SPARK_HADOOP_VERSION.tgz +ENV SPARK_HOME=$WORKDIR/spark-$SPARK_VERSION-bin-hadoop$SPARK_HADOOP_VERSION + +ADD https://packages.confluent.io/archive/${CONFLUENT_MAJOR_VERSION}/confluent-community-$CONFLUENT_VERSION-2.12.tar.gz $WORKDIR +RUN cd $WORKDIR \ + && tar -xf $WORKDIR/confluent-community-$CONFLUENT_VERSION-2.12.tar.gz -C $WORKDIR/ \ + && rm $WORKDIR/confluent-community-$CONFLUENT_VERSION-2.12.tar.gz +ENV CONFLUENT_HOME=$WORKDIR/confluent-$CONFLUENT_VERSION + +ADD https://d1i4a15mxbxib1.cloudfront.net/api/plugins/confluentinc/kafka-connect-hdfs/versions/$KAFKA_CONNECT_HDFS_VERSION/confluentinc-kafka-connect-hdfs-$KAFKA_CONNECT_HDFS_VERSION.zip $WORKDIR +RUN cd $WORKDIR \ + && mkdir $WORKDIR/kafka-connectors \ + && unzip $WORKDIR/confluentinc-kafka-connect-hdfs-$KAFKA_CONNECT_HDFS_VERSION.zip -d $WORKDIR/kafka-connectors/ \ + && rm $WORKDIR/confluentinc-kafka-connect-hdfs-$KAFKA_CONNECT_HDFS_VERSION.zip \ + && printf "\nplugin.path=$WORKDIR/kafka-connectors\n" >> $CONFLUENT_HOME/etc/kafka/connect-distributed.properties +ENV KAFKA_CONNECT_PLUGIN_PATH_LIB_PATH=$WORKDIR/kafka-connectors/confluentinc-kafka-connect-hdfs-$KAFKA_CONNECT_HDFS_VERSION/lib diff --git a/packaging/bundle-validation/Dockerfile-base b/packaging/bundle-validation/base/Dockerfile similarity index 96% rename from packaging/bundle-validation/Dockerfile-base rename to packaging/bundle-validation/base/Dockerfile index 4e730f7229113..1e5fdc493578c 100644 --- a/packaging/bundle-validation/Dockerfile-base +++ b/packaging/bundle-validation/base/Dockerfile @@ -16,7 +16,7 @@ # FROM adoptopenjdk/openjdk8:alpine -RUN apk add --no-cache --upgrade bash curl jq +RUN apk add --no-cache --upgrade bash curl jq openjdk11 openjdk17 --repository=https://dl-cdn.alpinelinux.org/alpine/v3.15/community RUN mkdir /opt/bundle-validation ENV WORKDIR=/opt/bundle-validation diff --git a/packaging/bundle-validation/base/build_flink1136hive313spark313.sh b/packaging/bundle-validation/base/build_flink1136hive313spark313.sh new file mode 100755 index 0000000000000..721515e867460 --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1136hive313spark313.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.13.6 \ + --build-arg SPARK_VERSION=3.1.3 \ + --build-arg SPARK_HADOOP_VERSION=2.7 \ + -t hudi-ci-bundle-validation-base:flink1136hive313spark313 . +docker image tag hudi-ci-bundle-validation-base:flink1136hive313spark313 apachehudi/hudi-ci-bundle-validation-base:flink1136hive313spark313 diff --git a/packaging/bundle-validation/base/build_flink1146hive313spark323.sh b/packaging/bundle-validation/base/build_flink1146hive313spark323.sh new file mode 100755 index 0000000000000..0a9bf8ae38e6d --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1146hive313spark323.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.14.6 \ + --build-arg SPARK_VERSION=3.2.3 \ + --build-arg SPARK_HADOOP_VERSION=2.7 \ + -t hudi-ci-bundle-validation-base:flink1146hive313spark323 . +docker image tag hudi-ci-bundle-validation-base:flink1146hive313spark323 apachehudi/hudi-ci-bundle-validation-base:flink1146hive313spark323 diff --git a/packaging/bundle-validation/base/build_flink1153hive313spark331.sh b/packaging/bundle-validation/base/build_flink1153hive313spark331.sh new file mode 100755 index 0000000000000..0b83a293b21ba --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1153hive313spark331.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.15.3 \ + --build-arg SPARK_VERSION=3.3.1 \ + --build-arg SPARK_HADOOP_VERSION=2 \ + -t hudi-ci-bundle-validation-base:flink1153hive313spark331 . +docker image tag hudi-ci-bundle-validation-base:flink1153hive313spark331 apachehudi/hudi-ci-bundle-validation-base:flink1153hive313spark331 diff --git a/packaging/bundle-validation/base/build_flink1153hive313spark332.sh b/packaging/bundle-validation/base/build_flink1153hive313spark332.sh new file mode 100755 index 0000000000000..182f99500dab1 --- /dev/null +++ b/packaging/bundle-validation/base/build_flink1153hive313spark332.sh @@ -0,0 +1,26 @@ +#!/bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +docker build \ + --build-arg HIVE_VERSION=3.1.3 \ + --build-arg FLINK_VERSION=1.15.3 \ + --build-arg SPARK_VERSION=3.3.2 \ + --build-arg SPARK_HADOOP_VERSION=2 \ + -t hudi-ci-bundle-validation-base:flink1153hive313spark332 . +docker image tag hudi-ci-bundle-validation-base:flink1153hive313spark332 apachehudi/hudi-ci-bundle-validation-base:flink1153hive313spark332 diff --git a/packaging/bundle-validation/ci_run.sh b/packaging/bundle-validation/ci_run.sh index e5543399f18bc..bef2ab3282c63 100755 --- a/packaging/bundle-validation/ci_run.sh +++ b/packaging/bundle-validation/ci_run.sh @@ -28,9 +28,10 @@ # and it contains the CI environment-specific variables. HUDI_VERSION=$1 +JAVA_RUNTIME_VERSION=$2 # choose versions based on build profiles -if [[ ${SPARK_PROFILE} == 'spark2.4' ]]; then +if [[ ${SPARK_RUNTIME} == 'spark2.4' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=2.3.9 DERBY_VERSION=10.10.2.0 @@ -40,7 +41,7 @@ if [[ ${SPARK_PROFILE} == 'spark2.4' ]]; then CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 IMAGE_TAG=flink1136hive239spark248 -elif [[ ${SPARK_PROFILE} == 'spark3.1' ]]; then +elif [[ ${SPARK_RUNTIME} == 'spark3.1.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 @@ -50,7 +51,7 @@ elif [[ ${SPARK_PROFILE} == 'spark3.1' ]]; then CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 IMAGE_TAG=flink1136hive313spark313 -elif [[ ${SPARK_PROFILE} == 'spark3.2' ]]; then +elif [[ ${SPARK_RUNTIME} == 'spark3.2.3' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 @@ -60,7 +61,7 @@ elif [[ ${SPARK_PROFILE} == 'spark3.2' ]]; then CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 IMAGE_TAG=flink1146hive313spark323 -elif [[ ${SPARK_PROFILE} == 'spark3.3' ]]; then +elif [[ ${SPARK_RUNTIME} == 'spark3.3.1' ]]; then HADOOP_VERSION=2.7.7 HIVE_VERSION=3.1.3 DERBY_VERSION=10.14.1.0 @@ -70,6 +71,16 @@ elif [[ ${SPARK_PROFILE} == 'spark3.3' ]]; then CONFLUENT_VERSION=5.5.12 KAFKA_CONNECT_HDFS_VERSION=10.1.13 IMAGE_TAG=flink1153hive313spark331 +elif [[ ${SPARK_RUNTIME} == 'spark3.3.2' ]]; then + HADOOP_VERSION=2.7.7 + HIVE_VERSION=3.1.3 + DERBY_VERSION=10.14.1.0 + FLINK_VERSION=1.15.3 + SPARK_VERSION=3.3.2 + SPARK_HADOOP_VERSION=2 + CONFLUENT_VERSION=5.5.12 + KAFKA_CONNECT_HDFS_VERSION=10.1.13 + IMAGE_TAG=flink1153hive313spark332 fi # Copy bundle jars to temp dir for mounting @@ -108,4 +119,4 @@ docker build \ # run validation script in docker docker run -v $TMP_JARS_DIR:/opt/bundle-validation/jars -v $TMP_DATA_DIR:/opt/bundle-validation/data \ - -i hudi-ci-bundle-validation:$IMAGE_TAG bash validate.sh + -i hudi-ci-bundle-validation:$IMAGE_TAG bash validate.sh $JAVA_RUNTIME_VERSION diff --git a/packaging/bundle-validation/validate.sh b/packaging/bundle-validation/validate.sh index fbc35ddb497cd..7a42601101716 100755 --- a/packaging/bundle-validation/validate.sh +++ b/packaging/bundle-validation/validate.sh @@ -25,6 +25,8 @@ # - /data/ ################################################################################################# +JAVA_RUNTIME_VERSION=$1 +DEFAULT_JAVA_HOME=${JAVA_HOME} WORKDIR=/opt/bundle-validation JARS_DIR=${WORKDIR}/jars # link the jar names to easier to use names @@ -36,6 +38,26 @@ ln -sf $JARS_DIR/hudi-utilities-slim*.jar $JARS_DIR/utilities-slim.jar ln -sf $JARS_DIR/hudi-kafka-connect-bundle*.jar $JARS_DIR/kafka-connect.jar ln -sf $JARS_DIR/hudi-metaserver-server-bundle*.jar $JARS_DIR/metaserver.jar +## +# Function to change Java runtime version by changing JAVA_HOME +## +change_java_runtime_version () { + if [[ ${JAVA_RUNTIME_VERSION} == 'openjdk11' ]]; then + echo "Change JAVA_HOME to /usr/lib/jvm/java-11-openjdk" + export JAVA_HOME=/usr/lib/jvm/java-11-openjdk + elif [[ ${JAVA_RUNTIME_VERSION} == 'openjdk17' ]]; then + echo "Change JAVA_HOME to /usr/lib/jvm/java-17-openjdk" + export JAVA_HOME=/usr/lib/jvm/java-17-openjdk + fi +} + +## +# Function to change Java runtime version to default Java 8 +## +use_default_java_runtime () { + echo "Use default java runtime under ${DEFAULT_JAVA_HOME}" + export JAVA_HOME=${DEFAULT_JAVA_HOME} +} ## # Function to test the spark & hadoop-mr bundles with hive sync. @@ -52,6 +74,7 @@ test_spark_hadoop_mr_bundles () { local DERBY_PID=$! $HIVE_HOME/bin/hiveserver2 --hiveconf hive.aux.jars.path=$JARS_DIR/hadoop-mr.jar & local HIVE_PID=$! + change_java_runtime_version echo "::warning::validate.sh Writing sample data via Spark DataSource and run Hive Sync..." $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar < $WORKDIR/spark_hadoop_mr/write.scala @@ -65,6 +88,7 @@ test_spark_hadoop_mr_bundles () { exit 1 fi echo "::warning::validate.sh Query and validate the results using HiveQL" + use_default_java_runtime # save HiveQL query results hiveqlresultsdir=/tmp/hadoop-mr-bundle/hiveql/trips/results mkdir -p $hiveqlresultsdir @@ -101,6 +125,7 @@ test_utilities_bundle () { fi OUTPUT_DIR=/tmp/hudi-utilities-test/ rm -r $OUTPUT_DIR + change_java_runtime_version echo "::warning::validate.sh running deltastreamer" $SPARK_HOME/bin/spark-submit \ --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer \ @@ -128,6 +153,7 @@ test_utilities_bundle () { echo "::debug::this is the shell command: $SHELL_COMMAND" LOGFILE="$WORKDIR/${FUNCNAME[0]}.log" $SHELL_COMMAND >> $LOGFILE + use_default_java_runtime if [ "$?" -ne 0 ]; then SHELL_RESULT=$(cat $LOGFILE | grep "Counts don't match") echo "::error::validate.sh $SHELL_RESULT" @@ -146,6 +172,7 @@ test_utilities_bundle () { ## test_flink_bundle() { export HADOOP_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath) + change_java_runtime_version $FLINK_HOME/bin/start-cluster.sh $FLINK_HOME/bin/sql-client.sh -j $JARS_DIR/flink.jar -f $WORKDIR/flink/insert.sql sleep 10 # for test stability @@ -153,6 +180,7 @@ test_flink_bundle() { local EXIT_CODE=$? $FLINK_HOME/bin/stop-cluster.sh unset HADOOP_CLASSPATH + use_default_java_runtime if [ "$EXIT_CODE" -ne 0 ]; then echo "::error::validate.sh Flink bundle validation failed." exit 1 @@ -173,6 +201,7 @@ test_flink_bundle() { # KAFKA_CONNECT_PLUGIN_PATH_LIB_PATH: path to install hudi-kafka-connect-bundle.jar ## test_kafka_connect_bundle() { + change_java_runtime_version KAFKA_CONNECT_JAR=$1 cp $KAFKA_CONNECT_JAR $KAFKA_CONNECT_PLUGIN_PATH_LIB_PATH $CONFLUENT_HOME/bin/zookeeper-server-start $CONFLUENT_HOME/etc/kafka/zookeeper.properties & @@ -188,6 +217,7 @@ test_kafka_connect_bundle() { $WORKDIR/kafka/consume.sh local EXIT_CODE=$? kill $ZOOKEEPER_PID $KAFKA_SERVER_PID $SCHEMA_REG_PID + use_default_java_runtime if [ "$EXIT_CODE" -ne 0 ]; then echo "::error::validate.sh Kafka Connect bundle validation failed." exit 1 @@ -214,6 +244,7 @@ test_metaserver_bundle () { $HIVE_HOME/bin/hiveserver2 --hiveconf hive.aux.jars.path=$JARS_DIR/hadoop-mr.jar & local HIVE_PID=$! + change_java_runtime_version echo "::warning::validate.sh Writing sample data via Spark DataSource." $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar < $WORKDIR/service/write.scala ls /tmp/hudi-bundles/tests/trips @@ -223,6 +254,7 @@ test_metaserver_bundle () { $SPARK_HOME/bin/spark-shell --jars $JARS_DIR/spark.jar < $WORKDIR/service/read.scala numRecords=$(cat /tmp/metaserver-bundle/sparkdatasource/trips/results/*.csv | wc -l) echo $numRecords + use_default_java_runtime if [ "$numRecords" -ne 10 ]; then echo "::error::validate.sh Metaserver bundle validation failed." exit 1 @@ -263,12 +295,14 @@ if [ "$?" -ne 0 ]; then fi echo "::warning::validate.sh done validating utilities slim bundle" -echo "::warning::validate.sh validating flink bundle" -test_flink_bundle -if [ "$?" -ne 0 ]; then - exit 1 +if [[ ${JAVA_RUNTIME_VERSION} == 'openjdk8' ]]; then + echo "::warning::validate.sh validating flink bundle" + test_flink_bundle + if [ "$?" -ne 0 ]; then + exit 1 + fi + echo "::warning::validate.sh done validating flink bundle" fi -echo "::warning::validate.sh done validating flink bundle" echo "::warning::validate.sh validating kafka connect bundle" test_kafka_connect_bundle $JARS_DIR/kafka-connect.jar diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml index f225ca4d6fa81..6f9a927b0621d 100644 --- a/packaging/hudi-aws-bundle/pom.xml +++ b/packaging/hudi-aws-bundle/pom.xml @@ -24,7 +24,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-cli-bundle/hudi-cli-with-bundle.sh b/packaging/hudi-cli-bundle/hudi-cli-with-bundle.sh index 55e7653870003..50880d9a01d2b 100755 --- a/packaging/hudi-cli-bundle/hudi-cli-with-bundle.sh +++ b/packaging/hudi-cli-bundle/hudi-cli-with-bundle.sh @@ -16,14 +16,33 @@ # See the License for the specific language governing permissions and # limitations under the License. +JAKARTA_EL_VERSION=3.0.3 DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" echo "DIR is ${DIR}" -CLI_BUNDLE_JAR=`ls $DIR/target/hudi-cli-bundle*.jar | grep -v source | grep -v javadoc` -SPARK_BUNDLE_JAR=`ls $DIR/../hudi-spark-bundle/target/hudi-spark*-bundle*.jar | grep -v source | grep -v javadoc` + +if [ -z "$CLI_BUNDLE_JAR" ]; then + echo "Inferring CLI_BUNDLE_JAR path assuming this script is under Hudi repo" + CLI_BUNDLE_JAR=`ls $DIR/target/hudi-cli-bundle*.jar | grep -v source | grep -v javadoc` +fi + +if [ -z "$SPARK_BUNDLE_JAR" ]; then + echo "Inferring SPARK_BUNDLE_JAR path assuming this script is under Hudi repo" + SPARK_BUNDLE_JAR=`ls $DIR/../hudi-spark-bundle/target/hudi-spark*-bundle*.jar | grep -v source | grep -v javadoc` +fi + +echo "CLI_BUNDLE_JAR: $CLI_BUNDLE_JAR" +echo "SPARK_BUNDLE_JAR: $SPARK_BUNDLE_JAR" + HUDI_CONF_DIR="${DIR}"/conf # hudi aux lib contains jakarta.el jars, which need to be put directly on class path HUDI_AUX_LIB="${DIR}"/auxlib +if [ ! -d $HUDI_AUX_LIB ]; then + echo "Downloading necessary auxiliary jars for Hudi CLI" + wget https://repo1.maven.org/maven2/org/glassfish/jakarta.el/$JAKARTA_EL_VERSION/jakarta.el-$JAKARTA_EL_VERSION.jar -P auxlib + wget https://repo1.maven.org/maven2/jakarta/el/jakarta.el-api/$JAKARTA_EL_VERSION/jakarta.el-api-$JAKARTA_EL_VERSION.jar -P auxlib +fi + . "${DIR}"/conf/hudi-env.sh if [ -z "$CLI_BUNDLE_JAR" ] || [ -z "$SPARK_BUNDLE_JAR" ]; then diff --git a/packaging/hudi-cli-bundle/pom.xml b/packaging/hudi-cli-bundle/pom.xml index df71bc9c0b796..5861f8a8f8837 100644 --- a/packaging/hudi-cli-bundle/pom.xml +++ b/packaging/hudi-cli-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -35,6 +35,7 @@ 2.0.2 3.21.0 2.6.2 + 2.8.0 @@ -108,6 +109,7 @@ jakarta.validation:jakarta.validation-api net.java.dev.jna:jna + org.apache.commons:commons-configuration2 org.apache.httpcomponents:httpclient org.apache.httpcomponents:httpcore org.apache.httpcomponents:fluent-hc @@ -276,6 +278,11 @@ httpclient ${http.version} + + org.apache.commons + commons-configuration2 + ${commons-configuration-2.version} + org.apache.logging.log4j log4j-1.2-api diff --git a/packaging/hudi-datahub-sync-bundle/pom.xml b/packaging/hudi-datahub-sync-bundle/pom.xml index 29e12624a89e6..8aac61e71cbae 100644 --- a/packaging/hudi-datahub-sync-bundle/pom.xml +++ b/packaging/hudi-datahub-sync-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml index b942eb1db3fe0..5e4c0a29e294c 100644 --- a/packaging/hudi-flink-bundle/pom.xml +++ b/packaging/hudi-flink-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -643,6 +643,12 @@ hudi-platform-service + + + deployArtifacts + true + + org.apache.hudi diff --git a/packaging/hudi-gcp-bundle/pom.xml b/packaging/hudi-gcp-bundle/pom.xml index 4ecf13831ad93..4d6b12c3291af 100644 --- a/packaging/hudi-gcp-bundle/pom.xml +++ b/packaging/hudi-gcp-bundle/pom.xml @@ -22,7 +22,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -95,9 +95,9 @@ 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 @@ -164,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 725d30162644b..e3ea70554b05d 100644 --- a/packaging/hudi-hadoop-mr-bundle/pom.xml +++ b/packaging/hudi-hadoop-mr-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml index aebb6aab1ad3f..809ad781a0254 100644 --- a/packaging/hudi-hive-sync-bundle/pom.xml +++ b/packaging/hudi-hive-sync-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -76,6 +76,10 @@ org.apache.parquet:parquet-avro commons-io:commons-io org.openjdk.jol:jol-core + + com.esotericsoftware:kryo-shaded + com.esotericsoftware:minlog + org.objenesis:objenesis @@ -87,6 +91,23 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. + + + com.esotericsoftware.kryo. + org.apache.hudi.com.esotericsoftware.kryo. + + + com.esotericsoftware.reflectasm. + org.apache.hudi.com.esotericsoftware.reflectasm. + + + com.esotericsoftware.minlog. + org.apache.hudi.com.esotericsoftware.minlog. + + + org.objenesis. + org.apache.hudi.org.objenesis. + false @@ -154,5 +175,13 @@ compile + + + com.esotericsoftware + kryo-shaded + ${kryo.shaded.version} + compile + + diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml index 7d6dce5aeea0b..9ec703bf7cbbc 100644 --- a/packaging/hudi-integ-test-bundle/pom.xml +++ b/packaging/hudi-integ-test-bundle/pom.xml @@ -17,7 +17,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -266,6 +266,10 @@ org.openjdk.jol. org.apache.hudi.org.openjdk.jol. + + org.eclipse.jetty. + org.apache.hudi.org.eclipse.jetty. + @@ -306,12 +310,6 @@ - - io.javalin - javalin - 2.8.0 - - io.dropwizard.metrics metrics-core @@ -357,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 2fe3b7c6e7966..5b76ec46a6c9b 100644 --- a/packaging/hudi-kafka-connect-bundle/pom.xml +++ b/packaging/hudi-kafka-connect-bundle/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-metaserver-server-bundle/pom.xml b/packaging/hudi-metaserver-server-bundle/pom.xml index b795a33bbf30a..4c47be94a73da 100644 --- a/packaging/hudi-metaserver-server-bundle/pom.xml +++ b/packaging/hudi-metaserver-server-bundle/pom.xml @@ -1,11 +1,27 @@ + hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml index 9ba08817f46c0..6c2d8676dcb7c 100644 --- a/packaging/hudi-presto-bundle/pom.xml +++ b/packaging/hudi-presto-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml index c9bf4371cc38c..8ffebeea2d514 100644 --- a/packaging/hudi-spark-bundle/pom.xml +++ b/packaging/hudi-spark-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -346,6 +346,12 @@ hudi-platform-service + + + deployArtifacts + true + + org.apache.hudi diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml index ae34d9a1092af..457ca5cfa1972 100644 --- a/packaging/hudi-timeline-server-bundle/pom.xml +++ b/packaging/hudi-timeline-server-bundle/pom.xml @@ -21,7 +21,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml index 700da0b3ea0c9..ab219cf90cef0 100644 --- a/packaging/hudi-trino-bundle/pom.xml +++ b/packaging/hudi-trino-bundle/pom.xml @@ -20,7 +20,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 @@ -68,6 +68,8 @@ org.apache.hudi:hudi-common + org.apache.hudi:hudi-client-common + org.apache.hudi:hudi-java-client org.apache.hudi:hudi-hadoop-mr @@ -175,6 +177,22 @@ hudi-hadoop-mr-bundle ${project.version} + + org.apache.hudi + hudi-client-common + ${project.version} + + + guava + com.google.guava + + + + + org.apache.hudi + hudi-java-client + ${project.version} + diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml index 2f3e9788f7cb7..a509c47471409 100644 --- a/packaging/hudi-utilities-bundle/pom.xml +++ b/packaging/hudi-utilities-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml index fa27ec03344cf..c61774c874700 100644 --- a/packaging/hudi-utilities-slim-bundle/pom.xml +++ b/packaging/hudi-utilities-slim-bundle/pom.xml @@ -19,7 +19,7 @@ hudi org.apache.hudi - 0.13.0-SNAPSHOT + 0.13.1 ../../pom.xml 4.0.0 diff --git a/pom.xml b/pom.xml index 9bf782ae5e7ba..0774e31949582 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ org.apache.hudi hudi pom - 0.13.0-SNAPSHOT + 0.13.1 Apache Hudi brings stream style processing on big data https://github.com/apache/hudi Hudi @@ -89,7 +89,7 @@ 0.15 1.7 3.0.0-M1 - 0.37.0 + 0.42.1 1.8 4.0.2 @@ -761,6 +761,7 @@ + com.google.protobuf:protoc:${proto.version} ${protoc.version} true @@ -954,6 +955,12 @@ spark-hive_${scala.binary.version} ${spark.version} provided + + + log4j + apache-log4j-extras + + org.apache.spark @@ -1443,6 +1450,10 @@ log4j log4j + + log4j + apache-log4j-extras + org.apache.hbase * @@ -1495,6 +1506,10 @@ log4j log4j + + log4j + apache-log4j-extras + org.apache.hbase * @@ -1871,6 +1886,19 @@ + + hudi-platform-service + + + deployArtifacts + true + + + + hudi-platform-service + packaging/hudi-metaserver-server-bundle + + integration-tests @@ -2170,6 +2198,7 @@ ${spark33.version} ${spark3.version} 3 + 2.12.15 ${scala12.version} 2.12 hudi-spark3.3.x @@ -2183,7 +2212,7 @@ bringing these file-formats as dependencies as well, we need to make sure that versions are synchronized to avoid classpath ambiguity --> 1.12.2 - 1.7.4 + 1.7.8 1.11.1 4.8 2.13.3 @@ -2294,6 +2323,7 @@ ${spark33.version} ${spark3.version} 3.3 + 2.12.15 ${scala12.version} 2.12 hudi-spark3.3.x @@ -2307,7 +2337,7 @@ bringing these file-formats as dependencies as well, we need to make sure that versions are synchronized to avoid classpath ambiguity --> 1.12.2 - 1.7.4 + 1.7.8 1.11.1 4.8 2.13.3 @@ -2417,13 +2447,6 @@ - - hudi-platform-service - - hudi-platform-service - packaging/hudi-metaserver-server-bundle - - diff --git a/rfc/rfc-51/rfc-51.md b/rfc/rfc-51/rfc-51.md index 18b76116c866d..29115b463445e 100644 --- a/rfc/rfc-51/rfc-51.md +++ b/rfc/rfc-51/rfc-51.md @@ -202,7 +202,7 @@ tblproperties ( ### How to infer CDC results -| `HoodieCDCInferCase` | Infer case details | Infer logic | Note | +| `HoodieCDCInferenceCase` | Infer case details | Infer logic | Note | |----------------------|---------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------| | `AS_IS` | CDC file written (suffix contains `-cdc`) alongside base files (COW) or log files (MOR) | CDC info will be extracted as is | the read-optimized way to read CDC | | `BASE_FILE_INSERT` | Base files were written to a new file group | All records (in the current commit): `op=I`, `before=null`, `after=` | on-the-fly inference | diff --git a/scripts/release/create_source_directory.sh b/scripts/release/create_source_directory.sh new file mode 100755 index 0000000000000..b03ca79ef0016 --- /dev/null +++ b/scripts/release/create_source_directory.sh @@ -0,0 +1,29 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +rsync -a \ + --exclude ".git" --exclude ".gitignore" --exclude ".gitattributes" --exclude ".travis.yml" \ + --exclude ".github" --exclude "target" \ + --exclude ".idea" --exclude "*.iml" --exclude ".DS_Store" --exclude "build-target" \ + --exclude "docs/content" --exclude ".rubydeps" \ + --exclude "rfc" \ + --exclude "docker/images" \ + . $1 \ No newline at end of file diff --git a/scripts/release/create_source_release.sh b/scripts/release/create_source_release.sh index bd37c80b1e2cd..93dde1bab8a09 100755 --- a/scripts/release/create_source_release.sh +++ b/scripts/release/create_source_release.sh @@ -66,14 +66,7 @@ mkdir -p ${RELEASE_DIR} git clone ${HUDI_DIR} ${CLONE_DIR} cd ${CLONE_DIR} -rsync -a \ - --exclude ".git" --exclude ".gitignore" --exclude ".gitattributes" --exclude ".travis.yml" \ - --exclude ".github" --exclude "target" \ - --exclude ".idea" --exclude "*.iml" --exclude ".DS_Store" --exclude "build-target" \ - --exclude "docs/content" --exclude ".rubydeps" \ - --exclude "rfc" \ - --exclude "docker/images" \ - . hudi-$RELEASE_VERSION +$CURR_DIR/release/create_source_directory.sh hudi-$RELEASE_VERSION tar czf ${RELEASE_DIR}/hudi-${RELEASE_VERSION}.src.tgz hudi-$RELEASE_VERSION gpg --armor --detach-sig ${RELEASE_DIR}/hudi-${RELEASE_VERSION}.src.tgz 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 c6677bc58c9d6..7d44e5ffa9671 100755 --- a/scripts/release/deploy_staging_jars.sh +++ b/scripts/release/deploy_staging_jars.sh @@ -36,34 +36,41 @@ if [ "$#" -gt "1" ]; then exit 1 fi -BUNDLE_MODULES=$(find -s packaging -name 'hudi-*-bundle' -type d) -BUNDLE_MODULES_EXCLUDED="-${BUNDLE_MODULES//$'\n'/,-}" - declare -a ALL_VERSION_OPTS=( -# upload all module jars and bundle jars -"-Dscala-2.11 -Dspark2.4 -pl $BUNDLE_MODULES_EXCLUDED" -"-Dscala-2.12 -Dspark2.4 -pl $BUNDLE_MODULES_EXCLUDED" -"-Dscala-2.12 -Dspark3.3 -pl $BUNDLE_MODULES_EXCLUDED" -"-Dscala-2.12 -Dspark3.2 -pl $BUNDLE_MODULES_EXCLUDED" -"-Dscala-2.12 -Dspark3.1" # this profile goes last in this section to ensure bundles use avro 1.8 - -# spark bundles -"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-spark-bundle -am" +# Upload Spark specific modules and bundle jars +# For Spark 2.4, Scala 2.11: +# hudi-spark-common_2.11 +# hudi-spark_2.11 +# hudi-spark2_2.11 +# hudi-utilities_2.11 +# hudi-cli-bundle_2.11 +# hudi-spark2.4-bundle_2.11 +# hudi-utilities-bundle_2.11 +# hudi-utilities-slim-bundle_2.11 +"-Dscala-2.11 -Dspark2.4 -pl hudi-spark-datasource/hudi-spark-common,hudi-spark-datasource/hudi-spark2,hudi-spark-datasource/hudi-spark,hudi-utilities,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle,packaging/hudi-utilities-bundle,packaging/hudi-utilities-slim-bundle -am" +# For Spark 2.4, Scala 2.12: +# hudi-spark2.4-bundle_2.12 "-Dscala-2.12 -Dspark2.4 -pl packaging/hudi-spark-bundle -am" -"-Dscala-2.12 -Dspark3.3 -pl packaging/hudi-spark-bundle -am" -"-Dscala-2.12 -Dspark3.2 -pl packaging/hudi-spark-bundle -am" -"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-spark-bundle -am" +# For Spark 3.2, Scala 2.12: +# hudi-spark3.2.x_2.12 +# hudi-spark3.2plus-common +# hudi-spark3.2-bundle_2.12 +"-Dscala-2.12 -Dspark3.2 -pl hudi-spark-datasource/hudi-spark3.2.x,hudi-spark-datasource/hudi-spark3.2plus-common,packaging/hudi-spark-bundle -am" +# For Spark 3.1, Scala 2.12: +# All other modules and bundles using avro 1.8 +"-Dscala-2.12 -Dspark3.1" +# For Spark 3.3, Scala 2.12: +# hudi-spark3.3.x_2.12 +# hudi-cli-bundle_2.12 +# hudi-spark3.3-bundle_2.12 +"-Dscala-2.12 -Dspark3.3 -pl hudi-spark-datasource/hudi-spark3.3.x,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am" -# spark bundles (legacy) (not overwriting previous uploads as these jar names are unique) +# Upload legacy Spark bundles (not overwriting previous uploads as these jar names are unique) "-Dscala-2.11 -Dspark2 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark-bundle_2.11 "-Dscala-2.12 -Dspark2 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark-bundle_2.12 "-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 - -# flink bundles (overwriting previous uploads) +# Upload Flink bundles (overwriting previous uploads) "-Dscala-2.12 -Dflink1.13 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" "-Dscala-2.12 -Dflink1.14 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" "-Dscala-2.12 -Dflink1.15 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am" @@ -100,11 +107,13 @@ fi COMMON_OPTIONS="-DdeployArtifacts=true -DskipTests -DretryFailedDeploymentCount=10" for v in "${ALL_VERSION_OPTS[@]}" do - # clean everything before any round of depoyment - $MVN clean + # TODO: consider cleaning all modules by listing directories instead of specifying profile + echo "Cleaning everything before any deployment" + $MVN clean $COMMON_OPTIONS ${v} echo "Building with options ${v}" - $MVN install "$COMMON_OPTIONS" "${v}" + $MVN install $COMMON_OPTIONS ${v} + echo "Deploying to repository.apache.org with version options ${v%-am}" # remove `-am` option to only deploy intended modules - $MVN deploy "$COMMON_OPTIONS" "${v%-am}" + $MVN deploy $COMMON_OPTIONS ${v%-am} done diff --git a/scripts/release/validate_source_binary_files.sh b/scripts/release/validate_source_binary_files.sh new file mode 100755 index 0000000000000..e4ce431c71b95 --- /dev/null +++ b/scripts/release/validate_source_binary_files.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# fail immediately +set -o errexit +set -o nounset + +echo "Checking for binary files in the source files" +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 "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 the source files? - [OK]\n" diff --git a/scripts/release/validate_source_copyright.sh b/scripts/release/validate_source_copyright.sh new file mode 100755 index 0000000000000..5176e2a07ed66 --- /dev/null +++ b/scripts/release/validate_source_copyright.sh @@ -0,0 +1,55 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +### Checking for DISCLAIMER +echo "Checking for DISCLAIMER" +disclaimerFile="./DISCLAIMER" +if [ -f "$disclaimerFile" ]; then + echo "DISCLAIMER file should not be present [ERROR]" + exit 1 +fi +echo -e "\t\tDISCLAIMER file exists ? [OK]\n" + +### Checking for LICENSE and NOTICE +echo "Checking for LICENSE and NOTICE" +licenseFile="./LICENSE" +noticeFile="./NOTICE" +if [ ! -f "$licenseFile" ]; then + echo "License file missing [ERROR]" + exit 1 +fi +echo -e "\t\tLicense file exists ? [OK]" + +if [ ! -f "$noticeFile" ]; then + echo "Notice file missing [ERROR]" + exit 1 +fi +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 '.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 '.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" diff --git a/scripts/release/validate_source_rat.sh b/scripts/release/validate_source_rat.sh new file mode 100755 index 0000000000000..504e008ce0f6e --- /dev/null +++ b/scripts/release/validate_source_rat.sh @@ -0,0 +1,24 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +echo "Running RAT Check" +(bash -c "mvn apache-rat:check -DdeployArtifacts=true") || (echo -e "\t\t Rat Check Failed. [ERROR]\n\t\t Please run with --verbose to get details\n" && exit 1) +echo -e "\t\tRAT Check Passed [OK]\n" diff --git a/scripts/release/validate_staged_bundles.sh b/scripts/release/validate_staged_bundles.sh index 25182786e4eee..3fd2ab351b1f2 100755 --- a/scripts/release/validate_staged_bundles.sh +++ b/scripts/release/validate_staged_bundles.sh @@ -29,6 +29,9 @@ 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-cli-bundle_2.11/${VERSION}/hudi-cli-bundle_2.11-${VERSION}.jar" +"${STAGING_REPO}/hudi-cli-bundle_2.12/${VERSION}/hudi-cli-bundle_2.12-${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 +42,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..01c3e265b8c30 100755 --- a/scripts/release/validate_staged_release.sh +++ b/scripts/release/validate_staged_release.sh @@ -88,16 +88,6 @@ else ARTIFACT_SUFFIX=${RELEASE_VERSION}-rc${RC_NUM} fi - -rm -rf $LOCAL_SVN_DIR -mkdir $LOCAL_SVN_DIR -cd $LOCAL_SVN_DIR - -echo "Downloading from svn co ${ROOT_SVN_URL}/${REPO_TYPE}/${HUDI_REPO}" - -(bash -c "svn co ${ROOT_SVN_URL}/${REPO_TYPE}/${HUDI_REPO} $REDIRECT") || (echo -e "\t\t Unable to checkout ${ROOT_SVN_URL}/${REPO_TYPE}/${HUDI_REPO} to $REDIRECT. Please run with --verbose to get details\n" && exit -1) - -echo "Validating hudi-${ARTIFACT_SUFFIX} with release type \"${REPO_TYPE}\"" if [ $RELEASE_TYPE == "release" ]; then ARTIFACT_PREFIX= elif [ $RELEASE_TYPE == "dev" ]; then @@ -106,7 +96,21 @@ else echo "Unexpected RELEASE_TYPE: $RELEASE_TYPE" exit 1; fi -cd ${HUDI_REPO}/${ARTIFACT_PREFIX}${ARTIFACT_SUFFIX} + +rm -rf $LOCAL_SVN_DIR +mkdir $LOCAL_SVN_DIR +cd $LOCAL_SVN_DIR + +echo "Current directory: `pwd`" + +FULL_SVN_URL=${ROOT_SVN_URL}/${REPO_TYPE}/${HUDI_REPO}/${ARTIFACT_PREFIX}${ARTIFACT_SUFFIX} + +echo "Downloading from svn co $FULL_SVN_URL" + +(bash -c "svn co $FULL_SVN_URL $REDIRECT") || (echo -e "\t\t Unable to checkout $FULL_SVN_URL to $REDIRECT. Please run with --verbose to get details\n" && exit -1) + +echo "Validating hudi-${ARTIFACT_SUFFIX} with release type \"${REPO_TYPE}\"" +cd ${ARTIFACT_PREFIX}${ARTIFACT_SUFFIX} $SHASUM hudi-${ARTIFACT_SUFFIX}.src.tgz > got.sha512 echo "Checking Checksum of Source Release" @@ -125,55 +129,13 @@ echo "Checking Signature" 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'` - -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' - exit 1 -fi -echo -e "\t\tNo Binary Files in Source Release? - [OK]\n" +$CURR_DIR/release/validate_source_binary_files.sh ### END: Binary Files Check -### Checking for DISCLAIMER -echo "Checking for DISCLAIMER" -disclaimerFile="./DISCLAIMER" -if [ -f "$disclaimerFile" ]; then - echo "DISCLAIMER file should not be present [ERROR]" - exit 1 -fi -echo -e "\t\tDISCLAIMER file exists ? [OK]\n" - -### Checking for LICENSE and NOTICE -echo "Checking for LICENSE and NOTICE" -licenseFile="./LICENSE" -noticeFile="./NOTICE" -if [ ! -f "$licenseFile" ]; then - echo "License file missing [ERROR]" - exit 1 -fi -echo -e "\t\tLicense file exists ? [OK]" - -if [ ! -f "$noticeFile" ]; then - echo "Notice file missing [ERROR]" - exit 1 -fi -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` -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)" - exit 1 -fi -echo -e "\t\tLicensing Check Passed [OK]\n" +### Checking for DISCLAIMER, LICENSE, NOTICE and source file license +$CURR_DIR/release/validate_source_copyright.sh ### Checking for RAT -echo "Running RAT Check" -(bash -c "mvn apache-rat:check $REDIRECT") || (echo -e "\t\t Rat Check Failed. [ERROR]\n\t\t Please run with --verbose to get details\n" && exit 1) -echo -e "\t\tRAT Check Passed [OK]\n" +$CURR_DIR/release/validate_source_rat.sh popd