diff --git a/.github/workflows/bot.yml b/.github/workflows/bot.yml index 20515f7c750ed..ebf3caccd9c62 100644 --- a/.github/workflows/bot.yml +++ b/.github/workflows/bot.yml @@ -18,16 +18,12 @@ jobs: include: - scala: "scala-2.11" spark: "spark2" - skipModules: "" - scala: "scala-2.11" spark: "spark2,spark-shade-unbundle-avro" - skipModules: "" - scala: "scala-2.12" spark: "spark3.1.x" - skipModules: "!hudi-spark-datasource/hudi-spark3" - scala: "scala-2.12" spark: "spark3.1.x,spark-shade-unbundle-avro" - skipModules: "!hudi-spark-datasource/hudi-spark3" - scala: "scala-2.12" spark: "spark3" - scala: "scala-2.12" @@ -44,5 +40,4 @@ jobs: env: SCALA_PROFILE: ${{ matrix.scala }} SPARK_PROFILE: ${{ matrix.spark }} - SKIP_MODULES: ${{ matrix.skipModules }} - run: mvn install -P "$SCALA_PROFILE,$SPARK_PROFILE" -pl "$SKIP_MODULES" -DskipTests=true -Dmaven.javadoc.skip=true -B -V + run: mvn install -P "$SCALA_PROFILE,$SPARK_PROFILE" -DskipTests=true -Dmaven.javadoc.skip=true -B -V diff --git a/azure-pipelines.yml b/azure-pipelines.yml index cd75e28dae24e..c2d5b510de2a3 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -179,6 +179,8 @@ stages: jdkVersionOption: '1.8' mavenOptions: '-Xmx2g $(MAVEN_OPTS)' - job: IT + displayName: IT modules + timeoutInMinutes: '90' steps: - task: AzureCLI@2 displayName: Prepare for IT diff --git a/docker/README.md b/docker/README.md index 19293de188448..226775184e84f 100644 --- a/docker/README.md +++ b/docker/README.md @@ -90,4 +90,98 @@ You can find more information on [Docker Hub Repositories Manual](https://docs.d ## Docker Demo Setup -Please refer to the [Docker Demo Docs page](https://hudi.apache.org/docs/docker_demo). \ No newline at end of file +Please refer to the [Docker Demo Docs page](https://hudi.apache.org/docs/docker_demo). + +## Building Multi-Arch Images + +NOTE: The steps below require some code changes. Support for multi-arch builds in a fully automated manner is being +tracked by [HUDI-3601](https://issues.apache.org/jira/browse/HUDI-3601). + +By default, the docker images are built for x86_64 (amd64) architecture. Docker `buildx` allows you to build multi-arch +images, link them together with a manifest file, and push them all to a registry – with a single command. Let's say we +want to build for arm64 architecture. First we need to ensure that `buildx` setup is done locally. Please follow the +below steps (referred from https://www.docker.com/blog/multi-arch-images): + +``` +# List builders +~ ❯❯❯ docker buildx ls +NAME/NODE DRIVER/ENDPOINT STATUS PLATFORMS +default * docker + default default running linux/amd64, linux/arm64, linux/arm/v7, linux/arm/v6 + +# If you are using the default builder, which is basically the old builder, then do following +~ ❯❯❯ docker buildx create --name mybuilder +mybuilder +~ ❯❯❯ docker buildx use mybuilder +~ ❯❯❯ docker buildx inspect --bootstrap +[+] Building 2.5s (1/1) FINISHED + => [internal] booting buildkit 2.5s + => => pulling image moby/buildkit:master 1.3s + => => creating container buildx_buildkit_mybuilder0 1.2s +Name: mybuilder +Driver: docker-container + +Nodes: +Name: mybuilder0 +Endpoint: unix:///var/run/docker.sock +Status: running + +Platforms: linux/amd64, linux/arm64, linux/arm/v7, linux/arm/v6 +``` + +Now goto `/docker/hoodie/hadoop` and change the `Dockerfile` to pull dependent images corresponding to +arm64. For example, in [base/Dockerfile](./hoodie/hadoop/base/Dockerfile) (which pulls jdk8 image), change the +line `FROM openjdk:8u212-jdk-slim-stretch` to `FROM arm64v8/openjdk:8u212-jdk-slim-stretch`. + +Then, from under `/docker/hoodie/hadoop` directory, execute the following command to build as well as +push the image to the dockerhub repo: + +``` +# Run under hoodie/hadoop, the is optional, "latest" by default +docker buildx build --platform -t /[:] --push + +# For example, to build base image +docker buildx build base --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-base:linux-arm64-0.10.1 --push +``` + +Once the base image is pushed then you could do something similar for other images. +Change [hive](./hoodie/hadoop/hive_base/Dockerfile) dockerfile to pull the base image with tag corresponding to +linux/arm64 platform. + +``` +# Change below line in the Dockerfile +FROM apachehudi/hudi-hadoop_${HADOOP_VERSION}-base:latest +# as shown below +FROM --platform=linux/arm64 apachehudi/hudi-hadoop_${HADOOP_VERSION}-base:linux-arm64-0.10.1 + +# and then build & push from under hoodie/hadoop dir +docker buildx build hive_base --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:linux-arm64-0.10.1 --push +``` + +Similarly, for images that are dependent on hive (e.g. [base spark](./hoodie/hadoop/spark_base/Dockerfile) +, [sparkmaster](./hoodie/hadoop/sparkmaster/Dockerfile), [sparkworker](./hoodie/hadoop/sparkworker/Dockerfile) +and [sparkadhoc](./hoodie/hadoop/sparkadhoc/Dockerfile)), change the corresponding Dockerfile to pull the base hive +image with tag corresponding to arm64. Then build and push using `docker buildx` command. + +For the sake of completeness, here is a [patch](https://gist.github.com/xushiyan/cec16585e884cf0693250631a1d10ec2) which +shows what changes to make in Dockerfiles (assuming tag is named `linux-arm64-0.10.1`), and below is the list +of `docker buildx` commands. + +``` +docker buildx build base --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-base:linux-arm64-0.10.1 --push +docker buildx build datanode --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-datanode:linux-arm64-0.10.1 --push +docker buildx build historyserver --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-history:linux-arm64-0.10.1 --push +docker buildx build hive_base --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3:linux-arm64-0.10.1 --push +docker buildx build namenode --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-namenode:linux-arm64-0.10.1 --push +docker buildx build prestobase --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-prestobase_0.217:linux-arm64-0.10.1 --push +docker buildx build spark_base --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3-sparkbase_2.4.4:linux-arm64-0.10.1 --push +docker buildx build sparkadhoc --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3-sparkadhoc_2.4.4:linux-arm64-0.10.1 --push +docker buildx build sparkmaster --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3-sparkmaster_2.4.4:linux-arm64-0.10.1 --push +docker buildx build sparkworker --platform linux/arm64 -t apachehudi/hudi-hadoop_2.8.4-hive_2.3.3-sparkworker_2.4.4:linux-arm64-0.10.1 --push +``` + +Once all the required images are pushed to the dockerhub repos, then we need to do one additional change +in [docker compose](./compose/docker-compose_hadoop284_hive233_spark244.yml) file. +Apply [this patch](https://gist.github.com/codope/3dd986de5e54f0650dd74b6032e4456c) to the docker compose file so +that [setup_demo](./setup_demo.sh) pulls images with the correct tag for arm64. And now we should be ready to run the +setup script and follow the docker demo. diff --git a/docker/demo/trino-batch1.commands b/docker/demo/trino-batch1.commands new file mode 100644 index 0000000000000..d89c19b0bf0bf --- /dev/null +++ b/docker/demo/trino-batch1.commands @@ -0,0 +1,23 @@ +/* + * 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. + */ + +select symbol, max(ts) from stock_ticks_cow group by symbol HAVING symbol = 'GOOG'; +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_cow where symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; diff --git a/docker/demo/trino-batch2-after-compaction.commands b/docker/demo/trino-batch2-after-compaction.commands new file mode 100644 index 0000000000000..da42b4728252d --- /dev/null +++ b/docker/demo/trino-batch2-after-compaction.commands @@ -0,0 +1,21 @@ +/* + * 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. + */ + +select symbol, max(ts) from stock_ticks_mor_ro group by symbol HAVING symbol = 'GOOG'; +select symbol, ts, volume, open, close from stock_ticks_mor_ro where symbol = 'GOOG'; diff --git a/docker/demo/trino-table-check.commands b/docker/demo/trino-table-check.commands new file mode 100644 index 0000000000000..4362d79fe770c --- /dev/null +++ b/docker/demo/trino-table-check.commands @@ -0,0 +1,20 @@ +/* + * 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. + */ + +show tables; diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java index 94e56102dbd55..8ab6c0ca4f992 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/MetadataCommand.java @@ -108,7 +108,9 @@ public String set(@CliOption(key = {"metadataDir"}, } @CliCommand(value = "metadata create", help = "Create the Metadata Table if it does not exist") - public String create() throws IOException { + public String create( + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master + ) throws IOException { HoodieCLI.getTableMetaClient(); Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); try { @@ -123,7 +125,7 @@ public String create() throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); HoodieWriteConfig writeConfig = getWriteConfig(); - initJavaSparkContext(); + initJavaSparkContext(Option.of(master)); SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); return String.format("Created Metadata Table in %s (duration=%.2f secs)", metadataPath, timer.endTimer() / 1000.0); } @@ -145,7 +147,8 @@ public String delete() throws Exception { } @CliCommand(value = "metadata init", help = "Update the metadata table from commits since the creation") - public String init(@CliOption(key = {"readonly"}, unspecifiedDefaultValue = "false", + public String init(@CliOption(key = "sparkMaster", unspecifiedDefaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master, + @CliOption(key = {"readonly"}, unspecifiedDefaultValue = "false", help = "Open in read-only mode") final boolean readOnly) throws Exception { HoodieCLI.getTableMetaClient(); Path metadataPath = new Path(getMetadataTableBasePath(HoodieCLI.basePath)); @@ -159,7 +162,7 @@ public String init(@CliOption(key = {"readonly"}, unspecifiedDefaultValue = "fal HoodieTimer timer = new HoodieTimer().startTimer(); if (!readOnly) { HoodieWriteConfig writeConfig = getWriteConfig(); - initJavaSparkContext(); + initJavaSparkContext(Option.of(master)); SparkHoodieBackedTableMetadataWriter.create(HoodieCLI.conf, writeConfig, new HoodieSparkEngineContext(jsc)); } @@ -191,9 +194,11 @@ public String stats() throws IOException { } @CliCommand(value = "metadata list-partitions", help = "List all partitions from metadata") - public String listPartitions() throws IOException { + public String listPartitions( + @CliOption(key = "sparkMaster", unspecifiedDefaultValue = SparkUtil.DEFAULT_SPARK_MASTER, help = "Spark master") final String master + ) throws IOException { HoodieCLI.getTableMetaClient(); - initJavaSparkContext(); + initJavaSparkContext(Option.of(master)); HoodieMetadataConfig config = HoodieMetadataConfig.newBuilder().enable(true).build(); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(new HoodieSparkEngineContext(jsc), config, HoodieCLI.basePath, "/tmp"); @@ -357,9 +362,9 @@ private HoodieWriteConfig getWriteConfig() { .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build(); } - private void initJavaSparkContext() { + private void initJavaSparkContext(Option userDefinedMaster) { if (jsc == null) { - jsc = SparkUtil.initJavaSparkConf(SparkUtil.getDefaultConf("HoodieCLI", Option.empty())); + jsc = SparkUtil.initJavaSparkConf(SparkUtil.getDefaultConf("HoodieCLI", userDefinedMaster)); } } } \ No newline at end of file diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java index d3f858439ce69..8b6d47b9a9aaf 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java @@ -23,20 +23,11 @@ import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.utils.InputStreamConsumer; import org.apache.hudi.cli.utils.SparkUtil; -import org.apache.hudi.client.SparkRDDWriteClient; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.index.HoodieIndex; - -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.launcher.SparkLauncher; import org.springframework.shell.core.CommandMarker; import org.springframework.shell.core.annotation.CliCommand; @@ -162,13 +153,4 @@ public String deleteSavepoint(@CliOption(key = {"commit"}, help = "Delete a save } return String.format("Savepoint \"%s\" deleted.", instantTime); } - - private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception { - HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build()) - .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(); - return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config); - } - } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java index 1ce6dfb288d62..889d7945b00c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java @@ -170,6 +170,7 @@ private void shutdownCallback(Function callback) { if (null != callback) { callback.apply(null != error); } + this.started = false; }); } 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 7b67ff54a2aa5..9bad2e3486e7f 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 @@ -18,6 +18,8 @@ package org.apache.hudi.client; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.async.AsyncArchiveService; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -45,6 +47,7 @@ import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; 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; @@ -73,9 +76,8 @@ import org.apache.hudi.table.action.rollback.RollbackUtils; import org.apache.hudi.table.action.savepoint.SavepointHelpers; import org.apache.hudi.table.marker.WriteMarkersFactory; - -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; +import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -107,15 +109,16 @@ public abstract class BaseHoodieWriteClient index; + private final SupportsUpgradeDowngrade upgradeDowngradeHelper; + private transient WriteOperationType operationType; + private transient HoodieWriteCommitCallback commitCallback; + protected final transient HoodieMetrics metrics; protected transient Timer.Context writeTimer = null; protected transient Timer.Context compactionTimer; protected transient Timer.Context clusteringTimer; - private transient WriteOperationType operationType; - private transient HoodieWriteCommitCallback commitCallback; protected transient AsyncCleanerService asyncCleanerService; protected transient AsyncArchiveService asyncArchiveService; protected final TransactionManager txnManager; @@ -125,25 +128,32 @@ public abstract class BaseHoodieWriteClient timelineService) { + public BaseHoodieWriteClient(HoodieEngineContext context, + HoodieWriteConfig writeConfig, + Option timelineService, + SupportsUpgradeDowngrade upgradeDowngradeHelper) { super(context, writeConfig, timelineService); this.metrics = new HoodieMetrics(config); this.index = createIndex(writeConfig); this.txnManager = new TransactionManager(config, fs); + this.upgradeDowngradeHelper = upgradeDowngradeHelper; } protected abstract HoodieIndex createIndex(HoodieWriteConfig writeConfig); @@ -291,7 +301,7 @@ public void bootstrap(Option> extraMetadata) { if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) { throw new HoodieException("Cannot bootstrap the table in multi-writer mode"); } - HoodieTable table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); + HoodieTable table = initTable(WriteOperationType.UPSERT, Option.ofNullable(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)); rollbackFailedBootstrap(); table.bootstrap(context, extraMetadata); } @@ -299,7 +309,7 @@ public void bootstrap(Option> extraMetadata) { /** * Main API to rollback failed bootstrap. */ - public void rollbackFailedBootstrap() { + protected void rollbackFailedBootstrap() { LOG.info("Rolling back pending bootstrap if present"); HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction(); @@ -628,7 +638,7 @@ public void deleteSavepoint(String savepointTime) { * @return true if the savepoint was restored to successfully */ public void restoreToSavepoint(String savepointTime) { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = initTable(WriteOperationType.UNKNOWN, Option.empty()); SavepointHelpers.validateSavepointPresence(table, savepointTime); restoreToInstant(savepointTime); SavepointHelpers.validateSavepointRestore(table, savepointTime); @@ -636,25 +646,11 @@ public void restoreToSavepoint(String savepointTime) { @Deprecated public boolean rollback(final String commitInstantTime) throws HoodieRollbackException { - HoodieTable table = createTable(config, hadoopConf); + HoodieTable table = initTable(WriteOperationType.UNKNOWN, Option.empty()); Option pendingRollbackInfo = getPendingRollbackInfo(table.getMetaClient(), commitInstantTime); return rollback(commitInstantTime, pendingRollbackInfo, false); } - /** - * @Deprecated - * Rollback the inflight record changes with the given commit time. This - * will be removed in future in favor of {@link BaseHoodieWriteClient#restoreToInstant(String)} - * Adding this api for backwards compatability. - * @param commitInstantTime Instant time of the commit - * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. - * @throws HoodieRollbackException if rollback cannot be performed successfully - */ - @Deprecated - public boolean rollback(final String commitInstantTime, boolean skipLocking) throws HoodieRollbackException { - return rollback(commitInstantTime, Option.empty(), skipLocking); - } - /** * @Deprecated * Rollback the inflight record changes with the given commit time. This @@ -711,7 +707,7 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime(); Timer.Context timerContext = metrics.getRollbackCtx(); try { - HoodieTable table = createTable(config, hadoopConf, config.isMetadataTableEnabled()); + HoodieTable table = initTable(WriteOperationType.UNKNOWN, Option.empty()); Option restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime); if (restorePlanOption.isPresent()) { HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime); @@ -839,11 +835,18 @@ public void archive() { * Provides a new commit time for a write operation (insert/update/delete). */ public String startCommit() { + HoodieTableMetaClient metaClient = createMetaClient(true); + return startCommit(metaClient.getCommitActionType(), metaClient); + } + + /** + * Provides a new commit time for a write operation (insert/update/delete/insert_overwrite/insert_overwrite_table) with specified action. + */ + public String startCommit(String actionType, HoodieTableMetaClient metaClient) { CleanerUtils.rollbackFailedWrites(config.getFailedWritesCleanPolicy(), HoodieTimeline.COMMIT_ACTION, () -> rollbackFailedWrites()); String instantTime = HoodieActiveTimeline.createNewInstantTime(); - HoodieTableMetaClient metaClient = createMetaClient(true); - startCommit(instantTime, metaClient.getCommitActionType(), metaClient); + startCommit(instantTime, actionType, metaClient); return instantTime; } @@ -954,8 +957,16 @@ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieT return inflightTimelineExcludeClusteringCommit; } - private Option getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) { - return getPendingRollbackInfos(metaClient).getOrDefault(commitToRollback, Option.empty()); + protected Option getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) { + return getPendingRollbackInfo(metaClient, commitToRollback, true); + } + + protected Option getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback, boolean ignoreCompactionAndClusteringInstants) { + return getPendingRollbackInfos(metaClient, ignoreCompactionAndClusteringInstants).getOrDefault(commitToRollback, Option.empty()); + } + + protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient) { + return getPendingRollbackInfos(metaClient, true); } /** @@ -963,20 +974,25 @@ private Option getPendingRollbackInfo(HoodieTableMeta * @param metaClient instance of {@link HoodieTableMetaClient} to use. * @return map of pending commits to be rolled-back instants to Rollback Instant and Rollback plan Pair. */ - protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient) { + protected Map> getPendingRollbackInfos(HoodieTableMetaClient metaClient, boolean ignoreCompactionAndClusteringInstants) { List instants = metaClient.getActiveTimeline().filterPendingRollbackTimeline().getInstants().collect(Collectors.toList()); Map> infoMap = new HashMap<>(); for (HoodieInstant instant : instants) { try { HoodieRollbackPlan rollbackPlan = RollbackUtils.getRollbackPlan(metaClient, instant); String action = rollbackPlan.getInstantToRollback().getAction(); - if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) { - boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action) - && ClusteringUtils.getClusteringPlan(metaClient, instant).isPresent(); - if (!isClustering) { - String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime(); - infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan))); + if (ignoreCompactionAndClusteringInstants) { + if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) { + boolean isClustering = HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action) + && ClusteringUtils.getClusteringPlan(metaClient, new HoodieInstant(true, rollbackPlan.getInstantToRollback().getAction(), + rollbackPlan.getInstantToRollback().getCommitTime())).isPresent(); + if (!isClustering) { + String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime(); + infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan))); + } } + } else { + infoMap.putIfAbsent(rollbackPlan.getInstantToRollback().getCommitTime(), Option.of(new HoodiePendingRollbackInfo(instant, rollbackPlan))); } } catch (IOException e) { LOG.warn("Fetching rollback plan failed for " + infoMap + ", skip the plan", e); @@ -988,7 +1004,7 @@ protected Map> getPendingRollbackInfos /** * Rollback all failed writes. */ - public Boolean rollbackFailedWrites() { + protected Boolean rollbackFailedWrites() { return rollbackFailedWrites(false); } @@ -996,7 +1012,7 @@ public Boolean rollbackFailedWrites() { * Rollback all failed writes. * @param skipLocking if this is triggered by another parent transaction, locking can be skipped. */ - public Boolean rollbackFailedWrites(boolean skipLocking) { + protected Boolean rollbackFailedWrites(boolean skipLocking) { HoodieTable table = createTable(config, hadoopConf); List instantsToRollback = getInstantsToRollback(table.getMetaClient(), config.getFailedWritesCleanPolicy(), Option.empty()); Map> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient()); @@ -1208,7 +1224,8 @@ protected Option inlineScheduleClustering(Option> ex } protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { - String commitTime = HoodieActiveTimeline.createNewInstantTime(); + Option pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false); + String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); table.rollback(context, commitTime, inflightInstant, false, false); table.getActiveTimeline().revertReplaceCommitInflightToRequested(inflightInstant); @@ -1246,17 +1263,79 @@ public HoodieMetrics getMetrics() { } /** - * Get HoodieTable and init {@link Timer.Context}. + * Instantiates engine-specific instance of {@link HoodieTable} as well as performs necessary + * bootstrapping operations (for ex, validating whether Metadata Table has to be bootstrapped) * - * @param operationType write operation type + * NOTE: THIS OPERATION IS EXECUTED UNDER LOCK, THEREFORE SHOULD AVOID ANY OPERATIONS + * NOT REQUIRING EXTERNAL SYNCHRONIZATION + * + * @param metaClient instance of {@link HoodieTableMetaClient} * @param instantTime current inflight instant time - * @return HoodieTable + * @return instantiated {@link HoodieTable} */ - protected abstract HoodieTable getTableAndInitCtx(WriteOperationType operationType, String instantTime); + protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime); /** - * Sets write schema from last instant since deletes may not have schema set in the config. + * Instantiates and initializes instance of {@link HoodieTable}, performing crucial bootstrapping + * operations such as: + * + * NOTE: This method is engine-agnostic and SHOULD NOT be overloaded, please check on + * {@link #doInitTable(HoodieTableMetaClient, Option)} instead + * + *
    + *
  • Checking whether upgrade/downgrade is required
  • + *
  • Bootstrapping Metadata Table (if required)
  • + *
  • Initializing metrics contexts
  • + *
*/ + protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { + HoodieTableMetaClient metaClient = createMetaClient(true); + // Setup write schemas for deletes + if (operationType == WriteOperationType.DELETE) { + setWriteSchemaForDeletes(metaClient); + } + + HoodieTable table; + + this.txnManager.beginTransaction(); + try { + tryUpgrade(metaClient, instantTime); + table = doInitTable(metaClient, instantTime); + } finally { + this.txnManager.endTransaction(); + } + + // Validate table properties + metaClient.validateTableProperties(config.getProps(), operationType); + // Make sure that FS View is in sync + table.getHoodieView().sync(); + + switch (operationType) { + case INSERT: + case INSERT_PREPPED: + case UPSERT: + case UPSERT_PREPPED: + case BULK_INSERT: + case BULK_INSERT_PREPPED: + case INSERT_OVERWRITE: + case INSERT_OVERWRITE_TABLE: + setWriteTimer(table); + break; + case CLUSTER: + clusteringTimer = metrics.getClusteringCtx(); + break; + case COMPACT: + compactionTimer = metrics.getCompactionCtx(); + break; + default: + } + + return table; + } + + /** + * Sets write schema from last instant since deletes may not have schema set in the config. + */ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { try { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); @@ -1301,4 +1380,33 @@ public void close() { this.heartbeatClient.stop(); this.txnManager.close(); } + + private void setWriteTimer(HoodieTable table) { + String commitType = table.getMetaClient().getCommitActionType(); + if (commitType.equals(HoodieTimeline.COMMIT_ACTION)) { + writeTimer = metrics.getCommitCtx(); + } else if (commitType.equals(HoodieTimeline.DELTA_COMMIT_ACTION)) { + writeTimer = metrics.getDeltaCommitCtx(); + } + } + + private void tryUpgrade(HoodieTableMetaClient metaClient, Option instantTime) { + UpgradeDowngrade upgradeDowngrade = + new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); + + if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) { + // Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits + List instantsToRollback = getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, instantTime); + + Map> pendingRollbacks = getPendingRollbackInfos(metaClient); + instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty())); + + rollbackFailedWrites(pendingRollbacks, true); + + new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper) + .run(HoodieTableVersion.current(), instantTime.orElse(null)); + + metaClient.reloadActiveTimeline(); + } + } } 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 72f8e29c9fa8e..e3c1f7f769c9d 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 @@ -78,7 +78,8 @@ public void startServer() throws IOException { .serverPort(writeConfig.getEmbeddedTimelineServerPort()) .numThreads(writeConfig.getEmbeddedTimelineServerThreads()) .compress(writeConfig.getEmbeddedTimelineServerCompressOutput()) - .async(writeConfig.getEmbeddedTimelineServerUseAsync()); + .async(writeConfig.getEmbeddedTimelineServerUseAsync()) + .refreshTimelineBasedOnLatestCommit(writeConfig.isRefreshTimelineServerBasedOnLatestCommit()); // Only passing marker-related write configs to timeline server // if timeline-server-based markers are used. if (writeConfig.getMarkersType() == MarkerType.TIMELINE_SERVER_BASED) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java index 41b1812c08151..36f9d169faa47 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java @@ -31,6 +31,7 @@ import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; import javax.annotation.Nonnull; + import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -94,6 +95,12 @@ public class HoodieClusteringConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Filter clustering partitions that matched regex pattern"); + public static final ConfigProperty PARTITION_SELECTED = ConfigProperty + .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "partition.selected") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("Partitions to run clustering"); + public static final ConfigProperty PLAN_STRATEGY_CLASS_NAME = ConfigProperty .key("hoodie.clustering.plan.strategy.class") .defaultValue(SPARK_SIZED_BASED_CLUSTERING_PLAN_STRATEGY) @@ -473,6 +480,11 @@ public Builder withClusteringPartitionRegexPattern(String pattern) { return this; } + public Builder withClusteringPartitionSelected(String partitionSelected) { + clusteringConfig.setValue(PARTITION_SELECTED, partitionSelected); + return this; + } + public Builder withClusteringSkipPartitionsFromLatest(int clusteringSkipPartitionsFromLatest) { clusteringConfig.setValue(PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, String.valueOf(clusteringSkipPartitionsFromLatest)); return this; 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 0aac9308da439..2570e204eef3a 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 @@ -250,7 +250,7 @@ public class HoodieCompactionConfig extends HoodieConfig { public static final ConfigProperty PRESERVE_COMMIT_METADATA = ConfigProperty .key("hoodie.compaction.preserve.commit.metadata") - .defaultValue(false) + .defaultValue(true) .sinceVersion("0.11.0") .withDocumentation("When rewriting data, preserves existing hoodie_commit_time"); 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 b7b410817b2fd..4202cbda72765 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 @@ -44,6 +44,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; 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.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsDatadogConfig; @@ -338,6 +339,11 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Timeline archiving removes older instants from the timeline, after each write operation, to minimize metadata overhead. " + "Controls whether or not, the write should be failed as well, if such archiving fails."); + public static final ConfigProperty REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty + .key("hoodie.refresh.timeline.server.based.on.latest.commit") + .defaultValue(false) + .withDocumentation("Refresh timeline in timeline server based on latest commit apart from timeline hash difference. By default (false), "); + public static final ConfigProperty INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = ConfigProperty .key("hoodie.consistency.check.initial_interval_ms") .defaultValue(2000L) @@ -1038,6 +1044,10 @@ public boolean isFailOnTimelineArchivingEnabled() { return getBoolean(FAIL_ON_TIMELINE_ARCHIVING_ENABLE); } + public boolean isRefreshTimelineServerBasedOnLatestCommit() { + return getBoolean(REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT); + } + public int getMaxConsistencyChecks() { return getInt(MAX_CONSISTENCY_CHECKS); } @@ -1051,7 +1061,7 @@ public int getMaxConsistencyCheckIntervalMs() { } public BulkInsertSortMode getBulkInsertSortMode() { - String sortMode = getString(BULK_INSERT_SORT_MODE); + String sortMode = getStringOrDefault(BULK_INSERT_SORT_MODE); return BulkInsertSortMode.valueOf(sortMode.toUpperCase()); } @@ -1072,8 +1082,7 @@ public EngineType getEngineType() { } public boolean populateMetaFields() { - return Boolean.parseBoolean(getStringOrDefault(HoodieTableConfig.POPULATE_META_FIELDS, - HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())); + return getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS); } /** @@ -1301,6 +1310,10 @@ public long getClusteringSmallFileLimit() { return getLong(HoodieClusteringConfig.PLAN_STRATEGY_SMALL_FILE_LIMIT); } + public String getClusteringPartitionSelected() { + return getString(HoodieClusteringConfig.PARTITION_SELECTED); + } + public String getClusteringPartitionFilterRegexPattern() { return getString(HoodieClusteringConfig.PARTITION_REGEX_PATTERN); } @@ -1480,6 +1493,10 @@ public boolean isMetadataIndexColumnStatsForAllColumnsEnabled() { return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled(); } + public int getColumnStatsIndexParallelism() { + return metadataConfig.getColumnStatsIndexParallelism(); + } + public int getBloomIndexKeysPerBucket() { return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET); } @@ -1536,7 +1553,8 @@ public double getParquetCompressionRatio() { } public CompressionCodecName getParquetCompressionCodec() { - return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME)); + String codecName = getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME); + return CompressionCodecName.fromConf(StringUtils.isNullOrEmpty(codecName) ? null : codecName); } public boolean parquetDictionaryEnabled() { 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 7eafe268ba8e8..a58e4d65d2879 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 @@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; import org.apache.hudi.common.model.HoodieDeltaWriteStat; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieLogFile; @@ -71,6 +72,9 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.accumulateColumnRanges; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.aggregateColumnStats; + /** * IO Operation to append data onto an existing file. */ @@ -320,7 +324,7 @@ private void updateWriteStatus(HoodieDeltaWriteStat stat, AppendResult result) { statuses.add(this.writeStatus); } - private void processAppendResult(AppendResult result) { + private void processAppendResult(AppendResult result, List recordList) { HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); if (stat.getPath() == null) { @@ -339,6 +343,19 @@ private void processAppendResult(AppendResult result) { updateWriteStatus(stat, result); } + if (config.isMetadataIndexColumnStatsForAllColumnsEnabled()) { + Map> columnRangeMap = stat.getRecordsStats().isPresent() + ? stat.getRecordsStats().get().getStats() : new HashMap<>(); + final String filePath = stat.getPath(); + // initialize map of column name to map of stats name to stats value + Map> columnToStats = new HashMap<>(); + writeSchemaWithMetaFields.getFields().forEach(field -> columnToStats.putIfAbsent(field.name(), new HashMap<>())); + // collect stats for columns at once per record and keep iterating through every record to eventually find col stats for all fields. + recordList.forEach(record -> aggregateColumnStats(record, writeSchemaWithMetaFields, columnToStats, config.isConsistentLogicalTimestampEnabled())); + writeSchemaWithMetaFields.getFields().forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats)); + stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap)); + } + resetWriteCounts(); assert stat.getRuntimeStats() != null; LOG.info(String.format("AppendHandle for partitionPath %s filePath %s, took %d ms.", partitionPath, @@ -376,7 +393,7 @@ protected void appendDataAndDeleteBlocks(Map header) if (blocks.size() > 0) { AppendResult appendResult = writer.appendBlocks(blocks); - processAppendResult(appendResult); + processAppendResult(appendResult, recordList); recordList.clear(); keysToDelete.clear(); } @@ -417,7 +434,9 @@ public List close() { writer = null; // update final size, once for all log files - for (WriteStatus status: statuses) { + // TODO we can actually deduce file size purely from AppendResult (based on offset and size + // of the appended block) + for (WriteStatus status : statuses) { long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath())); status.getStat().setFileSizeInBytes(logFileSize); } 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 096c257b1f797..3e7e0b16e2cf8 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 @@ -18,6 +18,10 @@ package org.apache.hudi.io; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.fs.FSUtils; @@ -35,11 +39,6 @@ import org.apache.hudi.io.storage.HoodieFileWriter; import org.apache.hudi.io.storage.HoodieFileWriterFactory; import org.apache.hudi.table.HoodieTable; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -235,20 +234,14 @@ protected void setupWriteStatus() throws IOException { stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT); stat.setFileId(writeStatus.getFileId()); stat.setPath(new Path(config.getBasePath()), path); - stat.setTotalWriteBytes(computeTotalWriteBytes()); - stat.setFileSizeInBytes(computeFileSizeInBytes()); stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords()); + + long fileSize = FSUtils.getFileSize(fs, path); + stat.setTotalWriteBytes(fileSize); + stat.setFileSizeInBytes(fileSize); + RuntimeStats runtimeStats = new RuntimeStats(); runtimeStats.setTotalCreateTime(timer.endTimer()); stat.setRuntimeStats(runtimeStats); } - - protected long computeTotalWriteBytes() throws IOException { - return FSUtils.getFileSize(fs, path); - } - - protected long computeFileSizeInBytes() throws IOException { - return FSUtils.getFileSize(fs, path); - } - } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java index 12d075e0cb532..bad822c8dbdbc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java @@ -19,14 +19,9 @@ package org.apache.hudi.io; import org.apache.hudi.common.bloom.BloomFilter; -import org.apache.hudi.common.bloom.BloomFilterTypeCode; -import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; @@ -39,8 +34,6 @@ import org.apache.log4j.Logger; import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -53,27 +46,13 @@ public class HoodieKeyLookupHandle exten private final BloomFilter bloomFilter; private final List candidateRecordKeys; - private final boolean useMetadataTableIndex; - private Option fileName = Option.empty(); private long totalKeysChecked; public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, Pair partitionPathFileIDPair) { - this(config, hoodieTable, partitionPathFileIDPair, Option.empty(), false); - } - - public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable hoodieTable, - Pair partitionPathFileIDPair, Option fileName, - boolean useMetadataTableIndex) { super(config, hoodieTable, partitionPathFileIDPair); this.candidateRecordKeys = new ArrayList<>(); this.totalKeysChecked = 0; - if (fileName.isPresent()) { - ValidationUtils.checkArgument(FSUtils.getFileId(fileName.get()).equals(getFileId()), - "File name '" + fileName.get() + "' doesn't match this lookup handle fileid '" + getFileId() + "'"); - this.fileName = fileName; - } - this.useMetadataTableIndex = useMetadataTableIndex; this.bloomFilter = getBloomFilter(); } @@ -81,25 +60,16 @@ private BloomFilter getBloomFilter() { BloomFilter bloomFilter = null; HoodieTimer timer = new HoodieTimer().startTimer(); try { - if (this.useMetadataTableIndex) { - ValidationUtils.checkArgument(this.fileName.isPresent(), - "File name not available to fetch bloom filter from the metadata table index."); - Option bloomFilterByteBuffer = - hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), fileName.get()); - if (!bloomFilterByteBuffer.isPresent()) { - throw new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight()); - } - bloomFilter = - new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(bloomFilterByteBuffer.get()).toString(), - BloomFilterTypeCode.DYNAMIC_V0); + if (config.isMetadataBloomFilterIndexEnabled()) { + bloomFilter = hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()) + .orElseThrow(() -> new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight())); } else { try (HoodieFileReader reader = createNewFileReader()) { bloomFilter = reader.readBloomFilter(); } } } catch (IOException e) { - throw new HoodieIndexException(String.format("Error reading bloom filter from %s/%s - %s", - getPartitionPathFileIDPair().getLeft(), this.fileName, e)); + throw new HoodieIndexException(String.format("Error reading bloom filter from %s", getPartitionPathFileIDPair()), e); } LOG.info(String.format("Read bloom filter from %s in %d ms", partitionPathFileIDPair, timer.endTimer())); return bloomFilter; 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 32d4ec2a6d794..d38f66a86f912 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 @@ -61,6 +61,8 @@ import java.util.Map; import java.util.Set; +import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD_POS; + @SuppressWarnings("Duplicates") /** * Handle to merge incoming records to those in storage. @@ -262,7 +264,7 @@ private boolean writeUpdateRecord(HoodieRecord hoodieRecord, GenericRecord ol isDelete = HoodieOperation.isDelete(hoodieRecord.getOperation()); } } - return writeRecord(hoodieRecord, indexedRecord, isDelete); + return writeRecord(hoodieRecord, indexedRecord, isDelete, oldRecord); } protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOException { @@ -272,16 +274,16 @@ protected void writeInsertRecord(HoodieRecord hoodieRecord) throws IOExceptio if (insertRecord.isPresent() && insertRecord.get().equals(IGNORE_RECORD)) { return; } - if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()))) { + if (writeRecord(hoodieRecord, insertRecord, HoodieOperation.isDelete(hoodieRecord.getOperation()), null)) { insertRecordsWritten++; } } protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { - return writeRecord(hoodieRecord, indexedRecord, false); + return writeRecord(hoodieRecord, indexedRecord, false, null); } - protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete) { + protected boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord, boolean isDelete, GenericRecord oldRecord) { Option recordMetadata = hoodieRecord.getData().getMetadata(); if (!partitionPath.equals(hoodieRecord.getPartitionPath())) { HoodieUpsertException failureEx = new HoodieUpsertException("mismatched partition path, record partition: " @@ -292,8 +294,10 @@ protected boolean writeRecord(HoodieRecord hoodieRecord, Option close(); public List writeStatuses() { 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 eee676822a8aa..2f4bca81b18dc 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 @@ -18,7 +18,6 @@ package org.apache.hudi.metadata; -import org.apache.avro.specific.SpecificRecordBase; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; @@ -56,14 +55,16 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieMetadataException; +import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -81,11 +82,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; -import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; /** @@ -121,7 +122,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta * @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 bootstrap operations + * @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 */ @@ -203,7 +204,7 @@ private void enablePartitions() { * @param metadataConfig - Table config * @param metaClient - Meta client for the metadata table * @param fsView - Metadata table filesystem view to use - * @param isBootstrapCompleted - Is metadata table bootstrap completed + * @param isBootstrapCompleted - Is metadata table initializing completed */ private void enablePartition(final MetadataPartitionType partitionType, final HoodieMetadataConfig metadataConfig, final Option metaClient, Option fsView, boolean isBootstrapCompleted) { @@ -319,13 +320,13 @@ public List getEnabledPartitionTypes() { /** * Initialize the metadata table if it does not exist. - * - * If the metadata table does not exist, then file and partition listing is used to bootstrap the table. + *

+ * If the metadata table does not exist, then file and partition listing is used to initialize the table. * * @param engineContext - * @param actionMetadata Action metadata types extending Avro generated SpecificRecordBase + * @param actionMetadata Action metadata types extending Avro generated SpecificRecordBase * @param inflightInstantTimestamp Timestamp of an instant in progress on the dataset. This instant is ignored - * while deciding to bootstrap the metadata table. + * while deciding to initialize the metadata table. */ protected abstract void initialize(HoodieEngineContext engineContext, Option actionMetadata, @@ -345,66 +346,70 @@ public void initTableMetadata() { } /** - * Bootstrap the metadata table if needed. + * Initialize the metadata table if needed. * - * @param engineContext - Engine context - * @param dataMetaClient - Meta client for the data table - * @param actionMetadata - Optional action metadata - * @param - Action metadata types extending Avro generated SpecificRecordBase - * @param inflightInstantTimestamp - Timestamp of an instant in progress on the dataset. This instant is ignored + * @param dataMetaClient - meta client for the data table + * @param actionMetadata - optional action metadata + * @param inflightInstantTimestamp - timestamp of an instant in progress on the dataset + * @param - action metadata types extending Avro generated SpecificRecordBase * @throws IOException */ - protected void bootstrapIfNeeded(HoodieEngineContext engineContext, - HoodieTableMetaClient dataMetaClient, - Option actionMetadata, - Option inflightInstantTimestamp) throws IOException { + protected void initializeIfNeeded(HoodieTableMetaClient dataMetaClient, + Option actionMetadata, + Option inflightInstantTimestamp) throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); - boolean rebootstrap = false; + boolean reInitialize = false; // If the un-synced instants have been archived, then - // the metadata table will need to be bootstrapped again. + // the metadata table will need to be initialized again. if (exists) { - final HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf.get()) .setBasePath(metadataWriteConfig.getBasePath()).build(); + + if (dataWriteConfig.getMetadataConfig().populateMetaFields() != metadataMetaClient.getTableConfig().populateMetaFields()) { + LOG.info("Re-initiating metadata table properties since populate meta fields have changed"); + metadataMetaClient = initializeMetaClient(dataWriteConfig.getMetadataConfig().populateMetaFields()); + } + final Option latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - rebootstrap = isBootstrapNeeded(latestMetadataInstant, actionMetadata); + reInitialize = isBootstrapNeeded(latestMetadataInstant, actionMetadata); } - if (rebootstrap) { + if (reInitialize) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.REBOOTSTRAP_STR, 1)); - LOG.info("Deleting Metadata Table directory so that it can be re-bootstrapped"); + LOG.info("Deleting Metadata Table directory so that it can be re-initialized"); dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath()), true); exists = false; } if (!exists) { // Initialize for the first time by listing partitions and files directly from the file system - if (bootstrapFromFilesystem(engineContext, dataMetaClient, inflightInstantTimestamp)) { + if (initializeFromFilesystem(dataMetaClient, inflightInstantTimestamp)) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); } } } /** - * Whether bootstrap operation needed for this metadata table. + * Whether initialize operation needed for this metadata table. *

* Rollback of the first commit would look like un-synced instants in the metadata table. - * Action metadata is needed to verify the instant time and avoid erroneous bootstrapping. + * Action metadata is needed to verify the instant time and avoid erroneous initializing. *

* TODO: Revisit this logic and validate that filtering for all * commits timeline is the right thing to do * - * @return True if the bootstrap is not needed, False otherwise + * @return True if the initialize is not needed, False otherwise */ private boolean isBootstrapNeeded(Option latestMetadataInstant, Option actionMetadata) { if (!latestMetadataInstant.isPresent()) { - LOG.warn("Metadata Table will need to be re-bootstrapped as no instants were found"); + LOG.warn("Metadata Table will need to be re-initialized as no instants were found"); return true; } @@ -417,7 +422,7 @@ private boolean isBootstrapNeeded(Option pendingDataInstant = dataMetaClient.getActiveTimeline() .getInstants().filter(i -> !i.isCompleted()) .filter(i -> !inflightInstantTimestamp.isPresent() || !i.getTimestamp().equals(inflightInstantTimestamp.get())) @@ -493,7 +494,7 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi if (!pendingDataInstant.isEmpty()) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BOOTSTRAP_ERR_STR, 1)); - LOG.warn("Cannot bootstrap metadata table as operation(s) are in progress on the dataset: " + LOG.warn("Cannot initialize metadata table as operation(s) are in progress on the dataset: " + Arrays.toString(pendingDataInstant.toArray())); return false; } @@ -504,30 +505,28 @@ private boolean bootstrapFromFilesystem(HoodieEngineContext engineContext, Hoodi .getReverseOrderedInstants().findFirst().map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime); - HoodieTableMetaClient.withPropertyBuilder() + initializeMetaClient(dataWriteConfig.getMetadataConfig().populateMetaFields()); + initTableMetadata(); + initializeEnabledFileGroups(dataMetaClient, createInstantTime); + + // During cold startup, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out + // of these large number of files and calling the existing update(HoodieCommitMetadata) function does not scale + // well. Hence, we have a special commit just for the initialization scenario. + initialCommit(createInstantTime); + return true; + } + + private HoodieTableMetaClient initializeMetaClient(boolean populatMetaFields) throws IOException { + return HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(tableName) .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName(HoodieMetadataPayload.class.getName()) .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) .setRecordKeyFields(RECORD_KEY_FIELD_NAME) - .setPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) + .setPopulateMetaFields(populatMetaFields) .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); - - initTableMetadata(); - initializeEnabledFileGroups(dataMetaClient, createInstantTime); - - // List all partitions in the basePath of the containing dataset - LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); - engineContext.setJobStatus(this.getClass().getSimpleName(), "Bootstrap: initializing metadata table by listing files and partitions"); - List dirInfoList = listAllPartitions(dataMetaClient); - - // During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these - // large number of files and calling the existing update(HoodieCommitMetadata) function does not scale well. - // Hence, we have a special commit just for the bootstrap scenario. - bootstrapCommit(dirInfoList, createInstantTime); - return true; } /** @@ -642,6 +641,14 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata } } + private MetadataRecordsGenerationParams getRecordsGenerationParams() { + return new MetadataRecordsGenerationParams( + dataMetaClient, enabledPartitionTypes, dataWriteConfig.getBloomFilterType(), + dataWriteConfig.getBloomIndexParallelism(), + dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(), + dataWriteConfig.getColumnStatsIndexParallelism()); + } + /** * Interface to assist in converting commit metadata to List of HoodieRecords to be written to metadata table. * Updates of different commit metadata uses the same method to convert to HoodieRecords and hence. @@ -672,8 +679,8 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co */ @Override public void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, - commitMetadata, dataMetaClient, dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(), instantTime), !isTableServiceAction); + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords( + engineContext, commitMetadata, instantTime, getRecordsGenerationParams()), !isTableServiceAction); } /** @@ -684,8 +691,8 @@ public void update(HoodieCommitMetadata commitMetadata, String instantTime, bool */ @Override public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { - processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, - cleanMetadata, dataMetaClient, instantTime), false); + processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, + cleanMetadata, getRecordsGenerationParams(), instantTime), false); } /** @@ -697,7 +704,7 @@ public void update(HoodieCleanMetadata cleanMetadata, String instantTime) { @Override public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, - enabledPartitionTypes, metadataMetaClient.getActiveTimeline(), restoreMetadata, dataMetaClient, instantTime, + metadataMetaClient.getActiveTimeline(), restoreMetadata, getRecordsGenerationParams(), instantTime, metadata.getSyncedInstantTime()), false); } @@ -723,8 +730,8 @@ public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) } Map> records = - HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, enabledPartitionTypes, - metadataMetaClient.getActiveTimeline(), rollbackMetadata, dataMetaClient, instantTime, + HoodieTableMetadataUtil.convertMetadataToRecords(engineContext, metadataMetaClient.getActiveTimeline(), + rollbackMetadata, getRecordsGenerationParams(), instantTime, metadata.getSyncedInstantTime(), wasSynced); commit(instantTime, records, false); } @@ -836,20 +843,29 @@ protected void cleanIfNecessary(BaseHoodieWriteClient writeClient, String instan } /** - * This is invoked to bootstrap metadata table for a dataset. Bootstrap Commit has special handling mechanism due to its scale compared to + * This is invoked to initialize metadata table for a dataset. Bootstrap Commit has special handling mechanism due to its scale compared to * other regular commits. - * */ - protected void bootstrapCommit(List partitionInfoList, String createInstantTime) { - List partitions = partitionInfoList.stream().map(p -> - p.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : p.getRelativePath()).collect(Collectors.toList()); - final int totalFiles = partitionInfoList.stream().mapToInt(p -> p.getTotalFiles()).sum(); + private void initialCommit(String createInstantTime) { + // List all partitions in the basePath of the containing dataset + LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); + engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions"); + + List partitionInfoList = listAllPartitions(dataMetaClient); + List partitions = new ArrayList<>(); + AtomicLong totalFiles = new AtomicLong(0); + Map> partitionToFilesMap = partitionInfoList.stream().map(p -> { + final String partitionName = HoodieTableMetadataUtil.getPartition(p.getRelativePath()); + partitions.add(partitionName); + totalFiles.addAndGet(p.getTotalFiles()); + return Pair.of(partitionName, p.getFileNameToSizeMap()); + }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)); final Map> partitionToRecordsMap = new HashMap<>(); // Record which saves the list of all partitions HoodieRecord allPartitionRecord = HoodieMetadataPayload.createPartitionListRecord(partitions); if (partitions.isEmpty()) { - // in case of bootstrapping of a fresh table, there won't be any partitions, but we need to make a boostrap commit + // in case of initializing of a fresh table, there won't be any partitions, but we need to make a boostrap commit final HoodieData allPartitionRecordsRDD = engineContext.parallelize( Collections.singletonList(allPartitionRecord), 1); partitionToRecordsMap.put(MetadataPartitionType.FILES, allPartitionRecordsRDD); @@ -857,7 +873,7 @@ protected void bootstrapCommit(List partitionInfoList, String cre return; } - HoodieData partitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); + HoodieData filesPartitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); if (!partitionInfoList.isEmpty()) { HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); @@ -869,29 +885,41 @@ protected void bootstrapCommit(List partitionInfoList, String cre // Record which saves files within a partition return HoodieMetadataPayload.createPartitionFilesRecord( - partitionInfo.getRelativePath().isEmpty() ? NON_PARTITIONED_NAME : partitionInfo.getRelativePath(), Option.of(validFileNameToSizeMap), Option.empty()); + HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); }); - partitionRecords = partitionRecords.union(fileListRecords); + filesPartitionRecords = filesPartitionRecords.union(fileListRecords); + } + ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); + + if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( + engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams(), createInstantTime); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, recordsRDD); + } + + if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( + engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams()); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, recordsRDD); } LOG.info("Committing " + partitions.size() + " partitions and " + totalFiles + " files to metadata"); - ValidationUtils.checkState(partitionRecords.count() == (partitions.size() + 1)); - partitionToRecordsMap.put(MetadataPartitionType.FILES, partitionRecords); commit(createInstantTime, partitionToRecordsMap, false); } /** * A class which represents a directory and the files and directories inside it. - * + *

* A {@code PartitionFileInfo} object saves the name of the partition and various properties requires of each file - * required for bootstrapping the metadata table. Saving limited properties reduces the total memory footprint when - * a very large number of files are present in the dataset being bootstrapped. + * required for initializing the metadata table. Saving limited properties reduces the total memory footprint when + * a very large number of files are present in the dataset being initialized. */ static class DirectoryInfo implements Serializable { // Relative path of the directory (relative to the base directory) private final String relativePath; // Map of filenames within this partition to their respective sizes - private HashMap filenameToSizeMap; + private final HashMap filenameToSizeMap; // List of directories within this partition private final List subDirectories = new ArrayList<>(); // Is this a hoodie partition 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 bb4ae962038fe..e5262ad6bb9f2 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 @@ -28,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -89,6 +90,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -478,7 +480,7 @@ public abstract Option scheduleRollback(HoodieEngineContext String instantTime, HoodieInstant instantToRollback, boolean skipTimelinePublish, boolean shouldRollbackUsingMarkers); - + /** * Rollback the (inflight/committed) record changes with the given commit time. *

@@ -519,14 +521,19 @@ public abstract Option scheduleRestore(HoodieEngineContext co
                                                     String restoreInstantTime,
                                                     String instantToRestore);
 
+  public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
+    rollbackInflightCompaction(inflightInstant, s -> Option.empty());
+  }
+
   /**
    * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file
    * to the .requested file.
    *
    * @param inflightInstant Inflight Compaction Instant
    */
-  public void rollbackInflightCompaction(HoodieInstant inflightInstant) {
-    String commitTime = HoodieActiveTimeline.createNewInstantTime();
+  public void rollbackInflightCompaction(HoodieInstant inflightInstant, Function> getPendingRollbackInstantFunc) {
+    final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry
+        -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime());
     scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers());
     rollback(context, commitTime, inflightInstant, false, false);
     getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java
index a63eb3badbe91..3a889de753d86 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanPartitionFilter.java
@@ -24,6 +24,7 @@
 import java.util.Comparator;
 import java.util.List;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Partition filter utilities. Currently, we support three mode:
@@ -58,11 +59,18 @@ private static List recentDaysFilter(List partitions, HoodieWrit
   }
 
   private static List selectedPartitionsFilter(List partitions, HoodieWriteConfig config) {
+    Stream filteredPartitions = partitions.stream();
+
     String beginPartition = config.getBeginPartitionForClustering();
+    if (beginPartition != null) {
+      filteredPartitions = filteredPartitions.filter(path -> path.compareTo(beginPartition) >= 0);
+    }
+
     String endPartition = config.getEndPartitionForClustering();
-    List filteredPartitions = partitions.stream()
-        .filter(path -> path.compareTo(beginPartition) >= 0 && path.compareTo(endPartition) <= 0)
-        .collect(Collectors.toList());
-    return filteredPartitions;
+    if (endPartition != null) {
+      filteredPartitions = filteredPartitions.filter(path -> path.compareTo(endPartition) <= 0);
+    }
+
+    return filteredPartitions.collect(Collectors.toList());
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java
index f4aaeee0c9db2..5d62ef390233f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/PartitionAwareClusteringPlanStrategy.java
@@ -35,6 +35,7 @@
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
+import java.util.Arrays;
 import java.util.List;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
@@ -72,8 +73,8 @@ public Option generateClusteringPlan() {
     HoodieWriteConfig config = getWriteConfig();
     List partitionPaths = FSUtils.getAllPartitionPaths(getEngineContext(), config.getMetadataConfig(), metaClient.getBasePath());
 
-    // get regex matched partitions if set
-    partitionPaths = getRegexPatternMatchedPartitions(config, partitionPaths);
+    // get matched partitions if set
+    partitionPaths = getMatchedPartitions(config, partitionPaths);
     // filter the partition paths if needed to reduce list status
     partitionPaths = filterPartitionPaths(partitionPaths);
 
@@ -113,6 +114,15 @@ public Option generateClusteringPlan() {
         .build());
   }
 
+  public List getMatchedPartitions(HoodieWriteConfig config, List partitionPaths) {
+    String partitionSelected = config.getClusteringPartitionSelected();
+    if (!StringUtils.isNullOrEmpty(partitionSelected)) {
+      return Arrays.asList(partitionSelected.split(","));
+    } else {
+      return getRegexPatternMatchedPartitions(config, partitionPaths);
+    }
+  }
+
   public List getRegexPatternMatchedPartitions(HoodieWriteConfig config, List partitionPaths) {
     String pattern = config.getClusteringPartitionFilterRegexPattern();
     if (!StringUtils.isNullOrEmpty(pattern)) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java
index e7a4170ec7871..87ee7d94723d9 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
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.table.action.rollback;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieRollbackRequest;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
@@ -27,16 +26,20 @@
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.IOType;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieRollbackException;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.marker.MarkerBasedRollbackUtils;
 import org.apache.hudi.table.marker.WriteMarkers;
+
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -111,16 +114,18 @@ protected HoodieRollbackRequest getRollbackRequestForAppend(String markerFilePat
     // NOTE: Since we're rolling back incomplete Delta Commit, it only could have appended its
     //       block to the latest log-file
     // TODO(HUDI-1517) use provided marker-file's path instead
-    HoodieLogFile latestLogFile = FSUtils.getLatestLogFile(table.getMetaClient().getFs(), partitionPath, fileId,
-        HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime).get();
-
-    // NOTE: Marker's don't carry information about the cumulative size of the blocks that have been appended,
-    //       therefore we simply stub this value.
-    Map logFilesWithBlocsToRollback =
-        Collections.singletonMap(latestLogFile.getFileStatus().getPath().toString(), -1L);
+    Option latestLogFileOption = FSUtils.getLatestLogFile(table.getMetaClient().getFs(), partitionPath, fileId,
+        HoodieFileFormat.HOODIE_LOG.getFileExtension(), baseCommitTime);
+    
+    Map logFilesWithBlocsToRollback = new HashMap<>();
+    if (latestLogFileOption.isPresent()) {
+      HoodieLogFile latestLogFile = latestLogFileOption.get();
+      // NOTE: Marker's don't carry information about the cumulative size of the blocks that have been appended,
+      //       therefore we simply stub this value.
+      logFilesWithBlocsToRollback = Collections.singletonMap(latestLogFile.getFileStatus().getPath().toString(), -1L);
+    }
 
     return new HoodieRollbackRequest(relativePartitionPath, fileId, baseCommitTime, Collections.emptyList(),
         logFilesWithBlocsToRollback);
   }
-
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SupportsUpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SupportsUpgradeDowngrade.java
index 5e6b9db913fa5..a30396b63ea40 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SupportsUpgradeDowngrade.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SupportsUpgradeDowngrade.java
@@ -23,10 +23,12 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
+import java.io.Serializable;
+
 /**
  * Interface for engine-specific logic needed for upgrade and downgrade actions.
  */
-public interface SupportsUpgradeDowngrade {
+public interface SupportsUpgradeDowngrade extends Serializable {
   /**
    * @param config  Write config.
    * @param context {@link HoodieEngineContext} instance to use.
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java
index a053a9611050c..440bc95615391 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/cluster/strategy/TestPartitionAwareClusteringPlanStrategy.java
@@ -71,7 +71,7 @@ public void testFilterPartitionPaths() {
     fakeTimeBasedPartitionsPath.add("20210719");
     fakeTimeBasedPartitionsPath.add("20210721");
 
-    List list = strategyTestRegexPattern.getRegexPatternMatchedPartitions(hoodieWriteConfig, fakeTimeBasedPartitionsPath);
+    List list = strategyTestRegexPattern.getMatchedPartitions(hoodieWriteConfig, fakeTimeBasedPartitionsPath);
     assertEquals(2, list.size());
     assertTrue(list.contains("20210721"));
     assertTrue(list.contains("20210723"));
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
index 3488a1365ce88..007ad290aadd9 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
@@ -97,7 +97,7 @@ public HoodieWriteableTestTable forCommit(String instantTime) {
     return (HoodieWriteableTestTable) super.forCommit(instantTime);
   }
 
-  public HoodieWriteableTestTable withInserts(String partition, String fileId, List records, TaskContextSupplier contextSupplier) throws Exception {
+  public Path withInserts(String partition, String fileId, List records, TaskContextSupplier contextSupplier) throws Exception {
     FileCreateUtils.createPartitionMetaFile(basePath, partition);
     String fileName = baseFileName(currentInstantTime, fileId);
 
@@ -151,7 +151,7 @@ public HoodieWriteableTestTable withInserts(String partition, String fileId, Lis
       }
     }
 
-    return this;
+    return baseFilePath;
   }
 
   public Map> withLogAppends(List records) throws Exception {
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 1f5d14af744fb..fb613309d3fa2 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
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.client;
 
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.async.AsyncCleanerService;
 import org.apache.hudi.client.common.HoodieFlinkEngineContext;
 import org.apache.hudi.common.data.HoodieList;
@@ -62,9 +64,6 @@
 import org.apache.hudi.table.upgrade.FlinkUpgradeDowngradeHelper;
 import org.apache.hudi.table.upgrade.UpgradeDowngrade;
 import org.apache.hudi.util.FlinkClientUtil;
-
-import com.codahale.metrics.Timer;
-import org.apache.hadoop.conf.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -93,7 +92,7 @@ public class HoodieFlinkWriteClient extends
   private Option metadataWriterOption = Option.empty();
 
   public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig) {
-    super(context, writeConfig);
+    super(context, writeConfig, FlinkUpgradeDowngradeHelper.getInstance());
     this.bucketToHandles = new HashMap<>();
   }
 
@@ -136,7 +135,7 @@ public void bootstrap(Option> extraMetadata) {
   @Override
   public List upsert(List> records, String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+        initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient());
     final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(records.get(0), getConfig(),
@@ -152,7 +151,7 @@ public List upsert(List> records, String instantTim
   public List upsertPreppedRecords(List> preppedRecords, String instantTime) {
     // only used for metadata table, the upsert happens in single thread
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+        initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient());
     final HoodieWriteHandle writeHandle = getOrCreateWriteHandle(preppedRecords.get(0), getConfig(),
@@ -164,7 +163,7 @@ public List upsertPreppedRecords(List> preppedRecor
   @Override
   public List insert(List> records, String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+        initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
     // create the write handle if not exists
@@ -187,7 +186,7 @@ public List insert(List> records, String instantTim
   public List insertOverwrite(
       List> records, final String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
+        initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient());
     // create the write handle if not exists
@@ -206,7 +205,7 @@ public List insertOverwrite(
    */
   public List insertOverwriteTable(
       List> records, final String instantTime) {
-    HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime);
+    HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient());
     // create the write handle if not exists
@@ -239,7 +238,7 @@ public List bulkInsertPreppedRecords(List> preppedR
   @Override
   public List delete(List keys, String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
+        initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime));
     preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient());
     HoodieWriteMetadata> result = table.delete(context, instantTime, keys);
     return postWrite(result, instantTime, table);
@@ -397,11 +396,9 @@ public HoodieWriteMetadata> cluster(final String clusteringIns
   }
 
   @Override
-  protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
-    HoodieTableMetaClient metaClient = createMetaClient(true);
-    new UpgradeDowngrade(metaClient, config, context, FlinkUpgradeDowngradeHelper.getInstance())
-        .run(HoodieTableVersion.current(), instantTime);
-    return getTableAndInitCtx(metaClient, operationType);
+  protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    return getHoodieTable();
   }
 
   /**
@@ -488,20 +485,6 @@ public void cleanHandlesGracefully() {
     return writeHandle;
   }
 
-  private HoodieTable>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
-    if (operationType == WriteOperationType.DELETE) {
-      setWriteSchemaForDeletes(metaClient);
-    }
-    // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieFlinkTable table = getHoodieTable();
-    if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
-      writeTimer = metrics.getCommitCtx();
-    } else {
-      writeTimer = metrics.getDeltaCommitCtx();
-    }
-    return table;
-  }
-
   public HoodieFlinkTable getHoodieTable() {
     return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context);
   }
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 275ab4f5e0a33..aeb546b0ca5c6 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
@@ -91,7 +91,7 @@ protected  void initialize(HoodieEngineContext eng
                                                            Option inflightInstantTimestamp) {
     try {
       if (enabled) {
-        bootstrapIfNeeded(engineContext, dataMetaClient, actionMetadata, inflightInstantTimestamp);
+        initializeIfNeeded(dataMetaClient, actionMetadata, inflightInstantTimestamp);
       }
     } catch (IOException e) {
       LOG.error("Failed to initialize metadata table. Disabling the writer.", e);
diff --git a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java
index de95520854646..29f0e03ae42b0 100644
--- a/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java
+++ b/hudi-client/hudi-flink-client/src/test/java/org/apache/hudi/testutils/HoodieFlinkWriteableTestTable.java
@@ -108,7 +108,8 @@ public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId
   }
 
   public HoodieFlinkWriteableTestTable withInserts(String partition, String fileId, List records) throws Exception {
-    return (HoodieFlinkWriteableTestTable) withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null));
+    withInserts(partition, fileId, records, new org.apache.hudi.client.FlinkTaskContextSupplier(null));
+    return this;
   }
 
   public Map> withLogAppends(List records) throws Exception {
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 f365f29329782..9de9298c25ae9 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
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.client;
 
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.client.common.HoodieJavaEngineContext;
 import org.apache.hudi.client.embedded.EmbeddedTimelineService;
 import org.apache.hudi.common.data.HoodieList;
@@ -30,7 +32,6 @@
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieNotSupportedException;
@@ -40,9 +41,7 @@
 import org.apache.hudi.table.HoodieJavaTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
-
-import com.codahale.metrics.Timer;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.table.upgrade.JavaUpgradeDowngradeHelper;
 
 import java.util.List;
 import java.util.Map;
@@ -52,14 +51,14 @@ public class HoodieJavaWriteClient extends
     BaseHoodieWriteClient>, List, List> {
 
   public HoodieJavaWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
-    super(context, clientConfig);
+    super(context, clientConfig, JavaUpgradeDowngradeHelper.getInstance());
   }
 
   public HoodieJavaWriteClient(HoodieEngineContext context,
                                HoodieWriteConfig writeConfig,
                                boolean rollbackPending,
                                Option timelineService) {
-    super(context, writeConfig, timelineService);
+    super(context, writeConfig, timelineService, JavaUpgradeDowngradeHelper.getInstance());
   }
 
   @Override
@@ -99,7 +98,7 @@ protected HoodieTable>, List, List upsert(List> records,
                                   String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+        initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient());
     HoodieWriteMetadata> result = table.upsert(context, instantTime, records);
@@ -113,7 +112,7 @@ public List upsert(List> records,
   public List upsertPreppedRecords(List> preppedRecords,
                                                 String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.UPSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords);
@@ -123,7 +122,7 @@ public List upsertPreppedRecords(List> preppedRecor
   @Override
   public List insert(List> records, String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+        initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
     HoodieWriteMetadata> result = table.insert(context, instantTime, records);
@@ -137,7 +136,7 @@ public List insert(List> records, String instantTim
   public List insertPreppedRecords(List> preppedRecords,
                                                 String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.INSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords);
@@ -169,7 +168,7 @@ public List bulkInsertPreppedRecords(List> preppedR
                                                     String instantTime,
                                                     Option>>> bulkInsertPartitioner) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.bulkInsertPrepped(context, instantTime, preppedRecords, bulkInsertPartitioner);
@@ -180,7 +179,7 @@ public List bulkInsertPreppedRecords(List> preppedR
   public List delete(List keys,
                                   String instantTime) {
     HoodieTable>, List, List> table =
-        getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
+        initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime));
     preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient());
     HoodieWriteMetadata> result = table.delete(context,instantTime, keys);
     return postWrite(result, instantTime, table);
@@ -233,23 +232,11 @@ public HoodieWriteMetadata> cluster(final String clusteringIns
   }
 
   @Override
-  protected HoodieTable>, List, List> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
-    HoodieTableMetaClient metaClient = createMetaClient(true);
+  protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) {
     // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
-    return getTableAndInitCtx(metaClient, operationType);
-  }
 
-  private HoodieTable>, List, List> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
-    if (operationType == WriteOperationType.DELETE) {
-      setWriteSchemaForDeletes(metaClient);
-    }
     // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieJavaTable table = HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient);
-    if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
-      writeTimer = metrics.getCommitCtx();
-    } else {
-      writeTimer = metrics.getDeltaCommitCtx();
-    }
-    return table;
+    return HoodieJavaTable.create(config, (HoodieJavaEngineContext) context, metaClient);
   }
+
 }
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/upgrade/JavaUpgradeDowngradeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/upgrade/JavaUpgradeDowngradeHelper.java
new file mode 100644
index 0000000000000..e1c44d0913318
--- /dev/null
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/upgrade/JavaUpgradeDowngradeHelper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.table.upgrade;
+
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+import org.apache.hudi.table.HoodieJavaTable;
+import org.apache.hudi.table.HoodieTable;
+
+/**
+ * Java upgrade and downgrade helper
+ */
+public class JavaUpgradeDowngradeHelper implements SupportsUpgradeDowngrade {
+
+  private static final JavaUpgradeDowngradeHelper SINGLETON_INSTANCE =
+      new JavaUpgradeDowngradeHelper();
+
+  private JavaUpgradeDowngradeHelper() {}
+
+  public static JavaUpgradeDowngradeHelper getInstance() {
+    return SINGLETON_INSTANCE;
+  }
+
+  @Override
+  public HoodieTable getTable(HoodieWriteConfig config, HoodieEngineContext context) {
+    return HoodieJavaTable.create(config, context);
+  }
+
+  @Override
+  public String getPartitionColumns(HoodieWriteConfig config) {
+    return config.getProps().getProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key());
+  }
+}
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 d51d25616c70d..ec1ecd6a0c291 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
@@ -18,15 +18,15 @@
 
 package org.apache.hudi.client;
 
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.client.embedded.EmbeddedTimelineService;
 import org.apache.hudi.client.utils.TransactionUtils;
-import org.apache.hudi.common.HoodiePendingRollbackInfo;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
 import org.apache.hudi.common.metrics.Registry;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
-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.model.HoodieRecordPayload;
@@ -35,7 +35,6 @@
 import org.apache.hudi.common.model.TableServiceType;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.HoodieTableVersion;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -55,10 +54,6 @@
 import org.apache.hudi.table.action.compact.CompactHelpers;
 import org.apache.hudi.table.marker.WriteMarkersFactory;
 import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper;
-import org.apache.hudi.table.upgrade.UpgradeDowngrade;
-
-import com.codahale.metrics.Timer;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.SparkConf;
@@ -94,7 +89,7 @@ public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeC
 
   public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig,
                              Option timelineService) {
-    super(context, writeConfig, timelineService);
+    super(context, writeConfig, timelineService, SparkUpgradeDowngradeHelper.getInstance());
   }
 
   /**
@@ -147,13 +142,13 @@ public JavaRDD> filterExists(JavaRDD> hoodieReco
    */
   @Override
   public void bootstrap(Option> extraMetadata) {
-    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+    initTable(WriteOperationType.UPSERT, Option.ofNullable(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)).bootstrap(context, extraMetadata);
   }
 
   @Override
   public JavaRDD upsert(JavaRDD> records, String instantTime) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+        initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient());
     HoodieWriteMetadata> result = table.upsert(context, instantTime, records);
@@ -166,7 +161,7 @@ public JavaRDD upsert(JavaRDD> records, String inst
   @Override
   public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, String instantTime) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.UPSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateUpsertSchema();
     preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords);
@@ -176,7 +171,7 @@ public JavaRDD upsertPreppedRecords(JavaRDD> preppe
   @Override
   public JavaRDD insert(JavaRDD> records, String instantTime) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+        initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
     HoodieWriteMetadata> result = table.insert(context,instantTime, records);
@@ -186,7 +181,7 @@ public JavaRDD insert(JavaRDD> records, String inst
   @Override
   public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, String instantTime) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.INSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords);
@@ -201,7 +196,7 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
   public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) {
-    HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
+    HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient());
     HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records);
@@ -216,7 +211,7 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
   public HoodieWriteResult insertOverwriteTable(JavaRDD> records, final String instantTime) {
-    HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime);
+    HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient());
     HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records);
@@ -231,7 +226,7 @@ public JavaRDD bulkInsert(JavaRDD> records, String
   @Override
   public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime);
+        initTable(WriteOperationType.BULK_INSERT, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.BULK_INSERT, table.getMetaClient());
     HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner);
@@ -241,7 +236,7 @@ public JavaRDD bulkInsert(JavaRDD> records, String
   @Override
   public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) {
     HoodieTable>, JavaRDD, JavaRDD> table =
-        getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
+        initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime));
     table.validateInsertSchema();
     preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient());
     HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner);
@@ -250,14 +245,14 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr
 
   @Override
   public JavaRDD delete(JavaRDD keys, String instantTime) {
-    HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
+    HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime));
     preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient());
     HoodieWriteMetadata> result = table.delete(context,instantTime, keys);
     return postWrite(result, instantTime, table);
   }
 
   public HoodieWriteResult deletePartitions(List partitions, String instantTime) {
-    HoodieTable>, JavaRDD, JavaRDD> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime);
+    HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE_PARTITION, Option.ofNullable(instantTime));
     preWrite(instantTime, WriteOperationType.DELETE_PARTITION, table.getMetaClient());
     HoodieWriteMetadata> result = table.deletePartitions(context, instantTime, partitions);
     return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
@@ -330,7 +325,7 @@ protected HoodieWriteMetadata> compact(String compactionIns
     HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
     HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
     if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
-      table.rollbackInflightCompaction(inflightInstant);
+      table.rollbackInflightCompaction(inflightInstant, commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false));
       table.getMetaClient().reloadActiveTimeline();
     }
     compactionTimer = metrics.getCompactionCtx();
@@ -420,34 +415,14 @@ private void updateTableMetadata(HoodieTable>, JavaRD
   }
 
   @Override
-  protected HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(WriteOperationType operationType,
-                                                                                                                  String instantTime) {
-    HoodieTableMetaClient metaClient = createMetaClient(true);
-    UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(
-        metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance());
-    try {
-      this.txnManager.beginTransaction();
-      if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) {
-        // Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits
-        List instantsToRollback = getInstantsToRollback(
-            metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime));
-        Map> pendingRollbacks = getPendingRollbackInfos(metaClient);
-        instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
-        this.rollbackFailedWrites(pendingRollbacks, true);
-        new UpgradeDowngrade(
-            metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance())
-            .run(HoodieTableVersion.current(), instantTime);
-        metaClient.reloadActiveTimeline();
-      }
-      // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation,
-      // if it didn't exist before
-      // See https://issues.apache.org/jira/browse/HUDI-3343 for more details
-      initializeMetadataTable(Option.of(instantTime));
-    } finally {
-      this.txnManager.endTransaction();
-    }
-    metaClient.validateTableProperties(config.getProps(), operationType);
-    return getTableAndInitCtx(metaClient, operationType, instantTime);
+  protected HoodieTable>, JavaRDD, JavaRDD> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) {
+    // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation,
+    // if it didn't exist before
+    // See https://issues.apache.org/jira/browse/HUDI-3343 for more details
+    initializeMetadataTable(instantTime);
+
+    // Create a Hoodie table which encapsulated the commits and files visible
+    return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, config.isMetadataTableEnabled());
   }
 
   /**
@@ -480,22 +455,6 @@ private void completeTableService(TableServiceType tableServiceType, HoodieCommi
     }
   }
 
-  private HoodieTable>, JavaRDD, JavaRDD> getTableAndInitCtx(
-      HoodieTableMetaClient metaClient, WriteOperationType operationType, String instantTime) {
-    if (operationType == WriteOperationType.DELETE) {
-      setWriteSchemaForDeletes(metaClient);
-    }
-    // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieSparkTable table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, config.isMetadataTableEnabled());
-    if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
-      writeTimer = metrics.getCommitCtx();
-    } else {
-      writeTimer = metrics.getDeltaCommitCtx();
-    }
-    table.getHoodieView().sync();
-    return table;
-  }
-
   @Override
   protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
     // Create a Hoodie table after startTxn which encapsulated the commits and files visible.
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
index 91d1f4e4e4fa2..54c1c9f5ac05d 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
@@ -18,6 +18,10 @@
 
 package org.apache.hudi.client.clustering.run.strategy;
 
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieClusteringGroup;
 import org.apache.hudi.avro.model.HoodieClusteringPlan;
@@ -25,7 +29,7 @@
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.client.utils.ConcatenatingIterator;
-import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.ClusteringOperation;
@@ -58,11 +62,6 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -246,21 +245,28 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js
    */
   private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext jsc,
                                                                 List clusteringOps) {
-    return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> {
-      List> iteratorsForPartition = new ArrayList<>();
-      clusteringOpsPartition.forEachRemaining(clusteringOp -> {
-        try {
-          Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema()));
-          HoodieFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()));
-          iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema));
-        } catch (IOException e) {
-          throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
-              + " and " + clusteringOp.getDeltaFilePaths(), e);
-        }
-      });
+    SerializableConfiguration hadoopConf = new SerializableConfiguration(getHoodieTable().getHadoopConf());
+    HoodieWriteConfig writeConfig = getWriteConfig();
+
+    // 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
+    return 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.getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath()));
+              iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema));
+            } catch (IOException e) {
+              throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
+                  + " and " + clusteringOp.getDeltaFilePaths(), e);
+            }
+          });
 
-      return new ConcatenatingIterator<>(iteratorsForPartition);
-    }).map(this::transform);
+          return new ConcatenatingIterator<>(iteratorsForPartition);
+        })
+        .map(record -> transform(record, writeConfig));
   }
 
   /**
@@ -279,12 +285,12 @@ private JavaRDD[] convertStreamToArray(Stream>
   /**
    * Transform IndexedRecord into HoodieRecord.
    */
-  private HoodieRecord transform(IndexedRecord indexedRecord) {
+  private static  HoodieRecord transform(IndexedRecord indexedRecord, HoodieWriteConfig writeConfig) {
     GenericRecord record = (GenericRecord) indexedRecord;
     Option keyGeneratorOpt = Option.empty();
-    if (!getWriteConfig().populateMetaFields()) {
+    if (!writeConfig.populateMetaFields()) {
       try {
-        keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(getWriteConfig().getProps())));
+        keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps()));
       } catch (IOException e) {
         throw new HoodieIOException("Only BaseKeyGenerators are supported when meta columns are disabled ", e);
       }
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
index 32bca55099eda..8a2958eab9da8 100644
--- 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
@@ -20,8 +20,7 @@
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.utils.LazyIterableIterator;
-import org.apache.hudi.common.bloom.BloomFilterTypeCode;
-import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+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;
@@ -37,8 +36,6 @@
 import org.apache.spark.api.java.function.Function2;
 import scala.Tuple2;
 
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -113,7 +110,7 @@ protected List computeNext() {
       }
 
       List> partitionNameFileNameList = new ArrayList<>(fileToKeysMap.keySet());
-      Map, ByteBuffer> fileToBloomFilterMap =
+      Map, BloomFilter> fileToBloomFilterMap =
           hoodieTable.getMetadataTable().getBloomFilters(partitionNameFileNameList);
 
       final AtomicInteger totalKeys = new AtomicInteger(0);
@@ -126,11 +123,7 @@ protected List computeNext() {
         if (!fileToBloomFilterMap.containsKey(partitionPathFileNamePair)) {
           throw new HoodieIndexException("Failed to get the bloom filter for " + partitionPathFileNamePair);
         }
-        final ByteBuffer fileBloomFilterByteBuffer = fileToBloomFilterMap.get(partitionPathFileNamePair);
-
-        HoodieDynamicBoundedBloomFilter fileBloomFilter =
-            new HoodieDynamicBoundedBloomFilter(StandardCharsets.UTF_8.decode(fileBloomFilterByteBuffer).toString(),
-                BloomFilterTypeCode.DYNAMIC_V0);
+        final BloomFilter fileBloomFilter = fileToBloomFilterMap.get(partitionPathFileNamePair);
 
         List candidateRecordKeys = new ArrayList<>();
         hoodieKeyList.forEach(hoodieKey -> {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java
index 521bdb20c58fc..4fdb6a6be3cba 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java
@@ -24,6 +24,7 @@
 import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.util.BaseFileUtils;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ParquetUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
@@ -304,21 +305,28 @@ public static void updateColumnStatsIndexFor(
       if (validIndexTables.isEmpty()) {
         finalColStatsIndexDf = newColStatsIndexDf;
       } else {
-        // NOTE: That Parquet schema might deviate from the original table schema (for ex,
-        //       by upcasting "short" to "integer" types, etc), and hence we need to re-adjust it
-        //       prior to merging, since merging might fail otherwise due to schemas incompatibility
-        finalColStatsIndexDf =
-            tryMergeMostRecentIndexTableInto(
-                sparkSession,
-                newColStatsIndexDf,
-                // Load current most recent col-stats-index table
-                sparkSession.read().load(
-                    new Path(indexFolderPath, validIndexTables.get(validIndexTables.size() - 1)).toString()
-                )
-            );
-
-        // Clean up all index tables (after creation of the new index)
-        tablesToCleanup.addAll(validIndexTables);
+        Path latestIndexTablePath = new Path(indexFolderPath, validIndexTables.get(validIndexTables.size() - 1));
+
+        Option> existingIndexTableOpt =
+            tryLoadExistingIndexTable(sparkSession, latestIndexTablePath);
+
+        if (!existingIndexTableOpt.isPresent()) {
+          finalColStatsIndexDf = newColStatsIndexDf;
+        } else {
+          // NOTE: That Parquet schema might deviate from the original table schema (for ex,
+          //       by upcasting "short" to "integer" types, etc), and hence we need to re-adjust it
+          //       prior to merging, since merging might fail otherwise due to schemas incompatibility
+          finalColStatsIndexDf =
+              tryMergeMostRecentIndexTableInto(
+                  sparkSession,
+                  newColStatsIndexDf,
+                  // Load current most recent col-stats-index table
+                  existingIndexTableOpt.get()
+              );
+
+          // Clean up all index tables (after creation of the new index)
+          tablesToCleanup.addAll(validIndexTables);
+        }
       }
 
       // Persist new col-stats-index table
@@ -349,6 +357,17 @@ public static void updateColumnStatsIndexFor(
     }
   }
 
+  @Nonnull
+  private static Option> tryLoadExistingIndexTable(@Nonnull SparkSession sparkSession, @Nonnull Path indexTablePath) {
+    try {
+      Dataset indexTableDataset = sparkSession.read().load(indexTablePath.toUri().toString());
+      return Option.of(indexTableDataset);
+    } catch (Exception e) {
+      LOG.error(String.format("Failed to load existing Column Stats index table from (%s)", indexTablePath), e);
+      return Option.empty();
+    }
+  }
+
   @Nonnull
   private static Dataset tryMergeMostRecentIndexTableInto(
       @Nonnull SparkSession sparkSession,
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 c905f92c2eac0..80b94edf7ecd6 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
@@ -113,7 +113,7 @@ protected  void initialize(HoodieEngineContext eng
       });
 
       if (enabled) {
-        bootstrapIfNeeded(engineContext, dataMetaClient, actionMetadata, inflightInstantTimestamp);
+        initializeIfNeeded(dataMetaClient, actionMetadata, inflightInstantTimestamp);
       }
     } catch (IOException e) {
       LOG.error("Failed to initialize metadata table. Disabling the writer.", e);
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
index 62bcbf684b836..69005cd75332c 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/AvroConversionUtils.scala
@@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.avro.SchemaConverters
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.encoders.RowEncoder
-import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType}
 import org.apache.spark.sql.{Dataset, Row, SparkSession}
 
 import scala.collection.JavaConversions._
@@ -144,7 +144,7 @@ object AvroConversionUtils {
   def convertStructTypeToAvroSchema(structType: DataType,
                                     structName: String,
                                     recordNamespace: String): Schema = {
-    getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace))
+    getAvroSchemaWithDefaults(SchemaConverters.toAvroType(structType, nullable = false, structName, recordNamespace), structType)
   }
 
   /**
@@ -154,13 +154,20 @@ object AvroConversionUtils {
     * @param schema     input avro schema
     * @return           Avro schema with null default set to nullable fields
     */
-  def getAvroSchemaWithDefaults(schema: Schema): Schema = {
+  def getAvroSchemaWithDefaults(schema: Schema, dataType: DataType): Schema = {
 
     schema.getType match {
       case Schema.Type.RECORD => {
-
+        val structType = dataType.asInstanceOf[StructType]
+        val structFields = structType.fields
         val modifiedFields = schema.getFields.map(field => {
-          val newSchema = getAvroSchemaWithDefaults(field.schema())
+          val i: Int = structType.fieldIndex(field.name())
+          val comment: String = if (structFields(i).metadata.contains("comment")) {
+            structFields(i).metadata.getString("comment")
+          } else {
+            field.doc()
+          }
+          val newSchema = getAvroSchemaWithDefaults(field.schema(), structFields(i).dataType)
           field.schema().getType match {
             case Schema.Type.UNION => {
               val innerFields = newSchema.getTypes
@@ -168,27 +175,27 @@ object AvroConversionUtils {
               if(containsNullSchema) {
                 // Need to re shuffle the fields in list because to set null as default, null schema must be head in union schema
                 val restructuredNewSchema = Schema.createUnion(List(Schema.create(Schema.Type.NULL)) ++ innerFields.filter(innerSchema => !(innerSchema.getType == Schema.Type.NULL)))
-                new Schema.Field(field.name(), restructuredNewSchema, field.doc(), JsonProperties.NULL_VALUE)
+                new Schema.Field(field.name(), restructuredNewSchema, comment, JsonProperties.NULL_VALUE)
               } else {
-                new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal())
+                new Schema.Field(field.name(), newSchema, comment, field.defaultVal())
               }
             }
-            case _ => new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal())
+            case _ => new Schema.Field(field.name(), newSchema, comment, field.defaultVal())
           }
         }).toList
         Schema.createRecord(schema.getName, schema.getDoc, schema.getNamespace, schema.isError, modifiedFields)
       }
 
       case Schema.Type.UNION => {
-        Schema.createUnion(schema.getTypes.map(innerSchema => getAvroSchemaWithDefaults(innerSchema)))
+        Schema.createUnion(schema.getTypes.map(innerSchema => getAvroSchemaWithDefaults(innerSchema, dataType)))
       }
 
       case Schema.Type.MAP => {
-        Schema.createMap(getAvroSchemaWithDefaults(schema.getValueType))
+        Schema.createMap(getAvroSchemaWithDefaults(schema.getValueType, dataType.asInstanceOf[MapType].valueType))
       }
 
       case Schema.Type.ARRAY => {
-        Schema.createArray(getAvroSchemaWithDefaults(schema.getElementType))
+        Schema.createArray(getAvroSchemaWithDefaults(schema.getElementType, dataType.asInstanceOf[ArrayType].elementType))
       }
 
       case _ => schema
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 b288289ac82ec..c963806416061 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
@@ -177,14 +177,24 @@ object HoodieSparkUtils extends SparkAdapterSupport {
    * Convert Filters to Catalyst Expressions and joined by And. If convert success return an
    * Non-Empty Option[Expression],or else return None.
    */
-  def convertToCatalystExpressions(filters: Array[Filter],
-                                   tableSchema: StructType): Option[Expression] = {
-    val expressions = filters.map(convertToCatalystExpression(_, tableSchema))
+  def convertToCatalystExpressions(filters: Seq[Filter],
+                                   tableSchema: StructType): Seq[Option[Expression]] = {
+    filters.map(convertToCatalystExpression(_, tableSchema))
+  }
+
+
+  /**
+   * Convert Filters to Catalyst Expressions and joined by And. If convert success return an
+   * Non-Empty Option[Expression],or else return None.
+   */
+  def convertToCatalystExpression(filters: Array[Filter],
+                                  tableSchema: StructType): Option[Expression] = {
+    val expressions = convertToCatalystExpressions(filters, tableSchema)
     if (expressions.forall(p => p.isDefined)) {
       if (expressions.isEmpty) {
         None
       } else if (expressions.length == 1) {
-        expressions(0)
+        expressions.head
       } else {
         Some(expressions.map(_.get).reduce(org.apache.spark.sql.catalyst.expressions.And))
       }
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
index 4cc1bca1d614d..fb6a5813ab9e0 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
@@ -27,7 +27,9 @@ import org.apache.spark.sql.hudi.SparkAdapter
 trait SparkAdapterSupport {
 
   lazy val sparkAdapter: SparkAdapter = {
-    val adapterClass = if (HoodieSparkUtils.isSpark3) {
+    val adapterClass = if (HoodieSparkUtils.gteqSpark3_2) {
+      "org.apache.spark.sql.adapter.Spark3_2Adapter"
+    } else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) {
       "org.apache.spark.sql.adapter.Spark3Adapter"
     } else {
       "org.apache.spark.sql.adapter.Spark2Adapter"
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
similarity index 75%
rename from hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala
rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
index 5c3035304cee7..4c4ddb5bf016c 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializerTrait.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroDeserializer.scala
@@ -24,12 +24,6 @@ package org.apache.spark.sql.avro
  *       If you're looking to convert Avro into "deserialized" [[Row]] (comprised of Java native types),
  *       please check [[AvroConversionUtils]]
  */
-trait HoodieAvroDeserializerTrait {
-  final def deserialize(data: Any): Option[Any] =
-    doDeserialize(data) match {
-      case opt: Option[_] => opt    // As of Spark 3.1, this will return data wrapped with Option, so we fetch the data
-      case row => Some(row)         // For other Spark versions, return the data as is
-    }
-
-  protected def doDeserialize(data: Any): Any
+trait HoodieAvroDeserializer {
+  def deserialize(data: Any): Option[Any]
 }
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
similarity index 97%
rename from hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala
rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
index 159d8da74d2db..84ba44b00fbbb 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializerTrait.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala
@@ -23,6 +23,6 @@ package org.apache.spark.sql.avro
  * NOTE: This is low-level component operating on Spark internal data-types (comprising [[InternalRow]]).
  *       If you're looking to convert "deserialized" [[Row]] into Avro, please check [[AvroConversionUtils]]
  */
-trait HoodieAvroSerializerTrait {
+trait HoodieAvroSerializer {
   def serialize(catalystData: Any): Any
 }
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 32ed2b16ce639..e41a9c1c8e326 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.hudi
 
 import org.apache.avro.Schema
 import org.apache.hudi.client.utils.SparkRowSerDe
-import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait}
+import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer}
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
@@ -43,16 +43,16 @@ import java.util.Locale
 trait SparkAdapter extends Serializable {
 
   /**
-   * Creates instance of [[HoodieAvroSerializerTrait]] providing for ability to serialize
+   * Creates instance of [[HoodieAvroSerializer]] providing for ability to serialize
    * Spark's [[InternalRow]] into Avro payloads
    */
-  def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait
+  def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer
 
   /**
-   * Creates instance of [[HoodieAvroDeserializerTrait]] providing for ability to deserialize
+   * Creates instance of [[HoodieAvroDeserializer]] providing for ability to deserialize
    * Avro payloads into Spark's [[InternalRow]]
    */
-  def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait
+  def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer
 
   /**
    * Create the SparkRowSerDe.
@@ -85,6 +85,16 @@ trait SparkAdapter extends Serializable {
   def getInsertIntoChildren(plan: LogicalPlan):
     Option[(LogicalPlan, Map[String, Option[String]], LogicalPlan, Boolean, Boolean)]
 
+  /**
+   * if the logical plan is a TimeTravelRelation LogicalPlan.
+   */
+  def isRelationTimeTravel(plan: LogicalPlan): Boolean
+
+  /**
+   * Get the member of the TimeTravelRelation LogicalPlan.
+   */
+  def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])]
+
   /**
    * Create a Insert Into LogicalPlan.
    */
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 223625fe7e469..1c40bc808c0be 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
@@ -328,6 +328,36 @@ public void testMetadataInsertUpsertClean(HoodieTableType tableType) throws Exce
     validateMetadata(testTable, emptyList(), true);
   }
 
+  @Test
+  public void testUpdationOfPopulateMetaFieldsForMetadataTable() throws Exception {
+    tableType = COPY_ON_WRITE;
+    init(tableType, false);
+
+    writeConfig = getWriteConfigBuilder(true, true, false)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withPopulateMetaFields(true)
+            .build())
+        .build();
+    initWriteConfigAndMetatableWriter(writeConfig, true);
+    doWriteOperation(testTable, "0000001", INSERT);
+
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(writeConfig.getBasePath() + "/.hoodie/metadata").setConf(hadoopConf).build();
+    assertTrue(metaClient.getTableConfig().populateMetaFields());
+
+    // update populateMeta fields to false.
+    writeConfig = getWriteConfigBuilder(true, true, false)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withPopulateMetaFields(false)
+            .build())
+        .build();
+    initWriteConfigAndMetatableWriter(writeConfig, true);
+    doWriteOperation(testTable, "0000002", INSERT);
+    metaClient = HoodieTableMetaClient.builder().setBasePath(writeConfig.getBasePath() + "/.hoodie/metadata").setConf(hadoopConf).build();
+    assertFalse(metaClient.getTableConfig().populateMetaFields());
+  }
+
   @Test
   public void testMetadataInsertUpsertCleanNonPartitioned() throws Exception {
     HoodieTableType tableType = COPY_ON_WRITE;
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 6ab33b422d0c1..6e67bd69bdfd7 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
@@ -56,6 +56,8 @@
 import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
 import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
+import org.apache.hudi.common.testutils.ClusteringTestUtils;
+import org.apache.hudi.common.testutils.FileCreateUtils;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.testutils.HoodieTestTable;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
@@ -1438,9 +1440,9 @@ public void testClusteringWithSortOneFilePerGroup(boolean populateMetaFields, bo
     testInsertAndClustering(clusteringConfig, populateMetaFields, true, true, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
   }
 
-  @ParameterizedTest
-  @MethodSource("populateMetaFieldsParams")
-  public void testPendingClusteringRollback(boolean populateMetaFields) throws Exception {
+  @Test
+  public void testPendingClusteringRollback() throws Exception {
+    boolean populateMetaFields = true;
     // setup clustering config.
     HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
         .withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).withInlineClustering(true).build();
@@ -1467,6 +1469,33 @@ public void testPendingClusteringRollback(boolean populateMetaFields) throws Exc
     metaClient.reloadActiveTimeline();
     // verify there are no pending clustering instants
     assertEquals(0, ClusteringUtils.getAllPendingClusteringPlans(metaClient).count());
+
+    // delete rollback.completed instant to mimic failed rollback of clustering. and then trigger rollback of clustering again. same rollback instant should be used.
+    HoodieInstant rollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get();
+    FileCreateUtils.deleteRollbackCommit(metaClient.getBasePath(), rollbackInstant.getTimestamp());
+    metaClient.reloadActiveTimeline();
+
+    // create replace commit requested meta file so that rollback will not throw FileNotFoundException
+    // create file slice with instantTime 001 and build clustering plan including this created 001 file slice.
+    HoodieClusteringPlan clusteringPlan = ClusteringTestUtils.createClusteringPlan(metaClient, pendingClusteringInstant.getTimestamp(), "1");
+    // create requested replace commit
+    HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
+        .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build();
+
+    FileCreateUtils.createRequestedReplaceCommit(metaClient.getBasePath(), pendingClusteringInstant.getTimestamp(), Option.of(requestedReplaceMetadata));
+
+    // trigger clustering again. no new rollback instants should be generated.
+    try {
+      client.cluster(pendingClusteringInstant.getTimestamp(), false);
+      // new replace commit metadata generated is fake one. so, clustering will fail. but the intention of test is ot check for duplicate rollback instants.
+    } catch (Exception e) {
+      //ignore.
+    }
+
+    metaClient.reloadActiveTimeline();
+    // verify that there is no new rollback instant generated
+    HoodieInstant newRollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get();
+    assertEquals(rollbackInstant.getTimestamp(), newRollbackInstant.getTimestamp());
   }
 
   @ParameterizedTest
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
index 2ff67c3c9156d..876a5d8dea12f 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.client.functional;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieAvroRecord;
@@ -47,7 +48,6 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner;
 import org.apache.hudi.testutils.Assertions;
-import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
 import org.apache.hudi.testutils.MetadataMergeWriteStatus;
 
@@ -63,6 +63,7 @@
 import org.junit.jupiter.params.provider.MethodSource;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
@@ -82,17 +83,24 @@
 import static org.junit.jupiter.api.Assertions.fail;
 
 @Tag("functional")
-public class TestHoodieIndex extends HoodieClientTestHarness {
+public class TestHoodieIndex extends TestHoodieMetadataBase {
 
   private static Stream indexTypeParams() {
+    // IndexType, populateMetaFields, enableMetadataIndex
     Object[][] data = new Object[][] {
-        {IndexType.BLOOM, true},
-        {IndexType.GLOBAL_BLOOM, true},
-        {IndexType.SIMPLE, true},
-        {IndexType.GLOBAL_SIMPLE, true},
-        {IndexType.SIMPLE, false},
-        {IndexType.GLOBAL_SIMPLE, false},
-        {IndexType.BUCKET, false}
+        {IndexType.BLOOM, true, true},
+        {IndexType.BLOOM, true, false},
+        {IndexType.GLOBAL_BLOOM, true, true},
+        {IndexType.GLOBAL_BLOOM, true, false},
+        {IndexType.SIMPLE, true, true},
+        {IndexType.SIMPLE, true, false},
+        {IndexType.SIMPLE, false, true},
+        {IndexType.SIMPLE, false, false},
+        {IndexType.GLOBAL_SIMPLE, true, true},
+        {IndexType.GLOBAL_SIMPLE, false, true},
+        {IndexType.GLOBAL_SIMPLE, false, false},
+        {IndexType.BUCKET, false, true},
+        {IndexType.BUCKET, false, false}
     };
     return Stream.of(data).map(Arguments::of);
   }
@@ -103,11 +111,11 @@ private static Stream indexTypeParams() {
   private HoodieIndex index;
   private HoodieWriteConfig config;
 
-  private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception {
-    setUp(indexType, populateMetaFields, true);
+  private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
+    setUp(indexType, populateMetaFields, true, enableMetadataIndex);
   }
 
-  private void setUp(IndexType indexType, boolean populateMetaFields, boolean rollbackUsingMarkers) throws Exception {
+  private void setUp(IndexType indexType, boolean populateMetaFields, boolean rollbackUsingMarkers, boolean enableMetadataIndex) throws Exception {
     this.indexType = indexType;
     initPath();
     initSparkContexts();
@@ -123,8 +131,13 @@ private void setUp(IndexType indexType, boolean populateMetaFields, boolean roll
         .withRollbackUsingMarkers(rollbackUsingMarkers)
         .withIndexConfig(indexBuilder.build())
         .withAutoCommit(false)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .withMetadataIndexBloomFilter(enableMetadataIndex)
+            .withMetadataIndexColumnStats(enableMetadataIndex)
+            .build())
         .withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps())
-            .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()).build();
+            .withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build())
+        .build();
     writeClient = getHoodieWriteClient(config);
     this.index = writeClient.getIndex();
   }
@@ -136,8 +149,8 @@ public void tearDown() throws IOException {
 
   @ParameterizedTest
   @MethodSource("indexTypeParams")
-  public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields) throws Exception {
-    setUp(indexType, populateMetaFields);
+  public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
+    setUp(indexType, populateMetaFields, enableMetadataIndex);
     String newCommitTime = "001";
     int totalRecords = 10 + random.nextInt(20);
     List records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -186,8 +199,8 @@ public void testSimpleTagLocationAndUpdate(IndexType indexType, boolean populate
 
   @ParameterizedTest
   @MethodSource("indexTypeParams")
-  public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields) throws Exception {
-    setUp(indexType, populateMetaFields);
+  public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
+    setUp(indexType, populateMetaFields, enableMetadataIndex);
     String newCommitTime = "001";
     int totalRecords = 10 + random.nextInt(20);
     List records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -236,8 +249,8 @@ public void testTagLocationAndDuplicateUpdate(IndexType indexType, boolean popul
 
   @ParameterizedTest
   @MethodSource("indexTypeParams")
-  public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception {
-    setUp(indexType, populateMetaFields, false);
+  public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
+    setUp(indexType, populateMetaFields, false, enableMetadataIndex);
     String newCommitTime = writeClient.startCommit();
     int totalRecords = 20 + random.nextInt(20);
     List records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -286,17 +299,21 @@ public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, bool
   }
 
   private static Stream regularIndexTypeParams() {
+    // IndexType, populateMetaFields, enableMetadataIndex
     Object[][] data = new Object[][] {
-        {IndexType.BLOOM, true},
-        {IndexType.SIMPLE, true}
+        // TODO (codope): Enabling metadata index is flaky. Both bloom_filter and col_stats get generated but loading column ranges from the index is failing.
+        // {IndexType.BLOOM, true, true},
+        {IndexType.BLOOM, true, false},
+        {IndexType.SIMPLE, true, true},
+        {IndexType.SIMPLE, true, false}
     };
     return Stream.of(data).map(Arguments::of);
   }
 
   @ParameterizedTest
   @MethodSource("regularIndexTypeParams")
-  public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean populateMetaFields) throws Exception {
-    setUp(indexType, populateMetaFields);
+  public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean populateMetaFields, boolean enableMetadataIndex) throws Exception {
+    setUp(indexType, populateMetaFields, enableMetadataIndex);
     String p1 = "2016/01/31";
     String p2 = "2015/01/31";
     String rowKey1 = UUID.randomUUID().toString();
@@ -320,7 +337,9 @@ public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean
     HoodieRecord record4 =
         new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
     JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
-
+    String newCommitTime = writeClient.startCommit();
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    writeClient.upsert(recordRDD, newCommitTime);
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
@@ -330,20 +349,42 @@ public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean
       assertFalse(record.isCurrentLocationKnown());
     }
 
-    // We create three parquet file, each having one record. (two different partitions)
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
-    String fileId1 = testTable.addCommit("001").getFileIdWithInserts(p1, record1);
-    String fileId2 = testTable.addCommit("002").getFileIdWithInserts(p1, record2);
-    String fileId3 = testTable.addCommit("003").getFileIdWithInserts(p2, record4);
+    // We create three parquet files, each having one record (two different partitions)
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
+    final String fileId1 = "fileID1";
+    final String fileId2 = "fileID2";
+    final String fileId3 = "fileID3";
+
+    Map>> partitionToFilesNameLengthMap = new HashMap<>();
+    Path baseFilePath = testTable.forCommit("0000001").withInserts(p1, fileId1, Collections.singletonList(record1));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p1, k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation("0000001", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
+        partitionToFilesNameLengthMap, false, false);
+
+    partitionToFilesNameLengthMap.clear();
+    baseFilePath = testTable.forCommit("0000002").withInserts(p1, fileId2, Collections.singletonList(record2));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p1, k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation("0000002", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
+        partitionToFilesNameLengthMap, false, false);
+
+    partitionToFilesNameLengthMap.clear();
+    baseFilePath = testTable.forCommit("0000003").withInserts(p2, fileId3, Collections.singletonList(record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p2, k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation("0000003", WriteOperationType.UPSERT, Arrays.asList(p1, p2),
+        partitionToFilesNameLengthMap, false, false);
 
     // We do the tag again
     metaClient = HoodieTableMetaClient.reload(metaClient);
     hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
+    List records = taggedRecordRDD.collect();
 
     // Check results
-    for (HoodieRecord record : taggedRecordRDD.collect()) {
+    for (HoodieRecord record : records) {
       if (record.getRecordKey().equals(rowKey1)) {
         if (record.getPartitionPath().equals(p2)) {
           assertEquals(record.getCurrentLocation().getFileId(), fileId3);
@@ -378,12 +419,17 @@ public void testTagLocationAndFetchRecordLocations(IndexType indexType, boolean
 
   @Test
   public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() throws Exception {
-    setUp(IndexType.GLOBAL_SIMPLE, true);
+    setUp(IndexType.GLOBAL_SIMPLE, true, true);
     config = getConfigBuilder()
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
             .withGlobalSimpleIndexUpdatePartitionPath(true)
             .withBloomIndexUpdatePartitionPath(true)
             .build())
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withMetadataIndexBloomFilter(true)
+            .withMetadataIndexColumnStats(true)
+            .build())
         .build();
     writeClient = getHoodieWriteClient(config);
     index = writeClient.getIndex();
@@ -432,7 +478,10 @@ public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath() thro
 
     final String file1P1C0 = UUID.randomUUID().toString();
     Map>> c1PartitionToFilesNameLengthMap = new HashMap<>();
-    c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, 100)));
+    // We have some records to be tagged (two different partitions)
+    Path baseFilePath = testTable.forCommit("1000").withInserts(p1, file1P1C0, Collections.singletonList(originalRecord));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    c1PartitionToFilesNameLengthMap.put(p1, Collections.singletonList(Pair.of(file1P1C0, Integer.valueOf((int) baseFileLength))));
     testTable.doWriteOperation("1000", WriteOperationType.INSERT, Arrays.asList(p1),
         c1PartitionToFilesNameLengthMap, false, false);
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
index 3141e1051ce5f..f00a0b8d19158 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
@@ -18,8 +18,8 @@
 
 package org.apache.hudi.client.functional;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieMetadataRecord;
+import org.apache.hudi.client.HoodieTimelineArchiver;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
@@ -52,8 +52,9 @@
 import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
 import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.client.HoodieTimelineArchiver;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
+
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.junit.jupiter.api.AfterEach;
@@ -437,5 +438,4 @@ protected HoodieWriteConfig getMetadataWriteConfig(HoodieWriteConfig writeConfig
     }
     return builder.build();
   }
-
 }
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 e61d6057cd80f..4421bd4d65442 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
@@ -18,8 +18,6 @@
 
 package org.apache.hudi.index.bloom;
 
-import org.apache.avro.Schema;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.functional.TestHoodieMetadataBase;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.bloom.BloomFilterFactory;
@@ -28,6 +26,7 @@
 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.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.testutils.RawTripTestPayload;
 import org.apache.hudi.common.util.Option;
@@ -37,10 +36,14 @@
 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.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.Path;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
 import org.junit.jupiter.api.AfterEach;
@@ -49,10 +52,11 @@
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
 import org.junit.jupiter.params.provider.MethodSource;
-import scala.Tuple2;
 
 import java.nio.file.Paths;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -61,6 +65,8 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import scala.Tuple2;
+
 import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -75,8 +81,13 @@ public class TestHoodieBloomIndex extends TestHoodieMetadataBase {
   private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
 
   public static Stream configParams() {
-    Object[][] data =
-        new Object[][]{{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}};
+    // rangePruning, treeFiltering, bucketizedChecking
+    Object[][] data = new Object[][] {
+        {true, true, true},
+        {false, true, true},
+        {true, true, false},
+        {true, false, true}
+    };
     return Stream.of(data).map(Arguments::of);
   }
 
@@ -87,6 +98,11 @@ public void setUp() throws Exception {
     initFileSystem();
     // We have some records to be tagged (two different partitions)
     initMetaClient();
+    HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM);
+    HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withIndexConfig(indexBuilder.build())
+        .build();
+    writeClient = getHoodieWriteClient(config);
   }
 
   @AfterEach
@@ -112,7 +128,7 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     HoodieBloomIndex index = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
 
     // Create some partitions, and put some files
     // "2016/01/21": 0 file
@@ -142,10 +158,40 @@ public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, b
     // Still 0, as no valid commit
     assertEquals(0, filesList.size());
 
-    testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
-    testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
-        .withInserts("2015/03/12", "3", record1)
-        .withInserts("2015/03/12", "4", record2, record3, record4);
+    final String fileId1 = "1";
+    final String fileId2 = "2";
+    final String fileId3 = "3";
+    final String fileId4 = "4";
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
+
+    String commitTime = "20160401010101";
+    Path baseFilePath = testTable.forCommit(commitTime).withInserts(partitions.get(1), fileId2, Collections.emptyList());
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(1),
+        k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(partitions.get(1)),
+        partitionToFilesNameLengthMap, false, false);
+
+    commitTime = "20150312101010";
+    partitionToFilesNameLengthMap.clear();
+    testTable.forCommit(commitTime);
+    baseFilePath = testTable.withInserts(partitions.get(2), fileId1, Collections.emptyList());
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+
+    baseFilePath = testTable.withInserts(partitions.get(2), fileId3, Collections.singletonList(record1));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
+        k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+
+    baseFilePath = testTable.withInserts(partitions.get(2), fileId4, Arrays.asList(record2, record3, record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partitions.get(2),
+        k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
+
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(partitions.get(2)),
+        partitionToFilesNameLengthMap, false, false);
 
     filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
     assertEquals(4, filesList.size());
@@ -229,9 +275,20 @@ public void testCheckUUIDsAgainstOneFile() throws Exception {
     // record2, record3).
     BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
     filter.add(record3.getRecordKey());
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter);
-    String fileId = testTable.addCommit("000").getFileIdWithInserts(partition, record1, record2);
-    String filename = testTable.getBaseFileNameById(fileId);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, filter, metadataWriter);
+
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
+    final String commitTime = "0000001";
+    final String fileId = UUID.randomUUID().toString();
+
+    Path baseFilePath = testTable.forCommit(commitTime)
+        .withInserts(partition, fileId, Arrays.asList(record1, record2));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition,
+        k -> new ArrayList<>()).add(Pair.of(fileId, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition),
+        partitionToFilesNameLengthMap, false, false);
+    final String filename = testTable.getBaseFileNameById(fileId);
 
     // The bloom filter contains 3 records
     assertTrue(filter.mightContain(record1.getRecordKey()));
@@ -305,7 +362,7 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean
     // Also create the metadata and config
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
 
     // Let's tag
     HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
@@ -316,10 +373,39 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean
       assertFalse(record.isCurrentLocationKnown());
     }
 
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
+    final String partition1 = "2016/01/31";
+    final String partition2 = "2015/01/31";
+
     // We create three parquet file, each having one record. (two different partitions)
-    String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1);
-    String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2);
-    String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4);
+    final String fileId1 = UUID.randomUUID().toString();
+    final String commit1 = "0000001";
+    Path baseFilePath = testTable.forCommit(commit1).withInserts(partition1, fileId1, Collections.singletonList(record1));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition1,
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit1, WriteOperationType.UPSERT, Collections.singletonList(partition1),
+        partitionToFilesNameLengthMap, false, false);
+
+    final String fileId2 = UUID.randomUUID().toString();
+    final String commit2 = "0000002";
+    baseFilePath = testTable.forCommit(commit2).withInserts(partition1, fileId2, Collections.singletonList(record2));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition1,
+        k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit2, WriteOperationType.UPSERT, Collections.singletonList(partition1),
+        partitionToFilesNameLengthMap, false, false);
+
+    final String fileId3 = UUID.randomUUID().toString();
+    final String commit3 = "0000003";
+    baseFilePath = testTable.forCommit(commit3).withInserts(partition2, fileId3, Collections.singletonList(record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition2,
+        k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit3, WriteOperationType.UPSERT, Collections.singletonList(partition2),
+        partitionToFilesNameLengthMap, false, false);
 
     // We do the tag again
     taggedRecordRDD = tagLocation(bloomIndex, recordRDD, HoodieSparkTable.create(config, context, metaClient));
@@ -327,7 +413,7 @@ public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean
     // Check results
     for (HoodieRecord record : taggedRecordRDD.collect()) {
       if (record.getRecordKey().equals(rowKey1)) {
-        if (record.getPartitionPath().equals("2015/01/31")) {
+        if (record.getPartitionPath().equals(partition2)) {
           assertEquals(record.getCurrentLocation().getFileId(), fileId3);
         } else {
           assertEquals(record.getCurrentLocation().getFileId(), fileId1);
@@ -370,7 +456,7 @@ public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean
     // Also create the metadata and config
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
 
     // Let's tag
     HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
@@ -387,10 +473,38 @@ public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean
       assertTrue(!record._2.isPresent());
     }
 
+    final String partition1 = "2016/01/31";
+    final String partition2 = "2015/01/31";
+    final String fileId1 = UUID.randomUUID().toString();
+    final String fileId2 = UUID.randomUUID().toString();
+    final String fileId3 = UUID.randomUUID().toString();
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
     // We create three parquet file, each having one record. (two different partitions)
-    String fileId1 = testTable.addCommit("001").getFileIdWithInserts("2016/01/31", record1);
-    String fileId2 = testTable.addCommit("002").getFileIdWithInserts("2016/01/31", record2);
-    String fileId3 = testTable.addCommit("003").getFileIdWithInserts("2015/01/31", record4);
+    final String commit1 = "0000001";
+    Path baseFilePath = testTable.forCommit(commit1).withInserts(partition1, fileId1, Collections.singletonList(record1));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition1,
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit1, WriteOperationType.UPSERT, Collections.singletonList(partition1),
+        partitionToFilesNameLengthMap, false, false);
+
+    final String commit2 = "0000002";
+    partitionToFilesNameLengthMap.clear();
+    baseFilePath = testTable.forCommit(commit2).withInserts(partition1, fileId2, Collections.singletonList(record2));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition1,
+        k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit2, WriteOperationType.UPSERT, Collections.singletonList(partition1),
+        partitionToFilesNameLengthMap, false, false);
+
+    final String commit3 = "0000003";
+    partitionToFilesNameLengthMap.clear();
+    baseFilePath = testTable.forCommit(commit3).withInserts(partition2, fileId3, Collections.singletonList(record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition2,
+        k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commit3, WriteOperationType.UPSERT, Collections.singletonList(partition2),
+        partitionToFilesNameLengthMap, false, false);
 
     // We do the tag again
     metaClient = HoodieTableMetaClient.reload(metaClient);
@@ -409,7 +523,7 @@ public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean
         assertEquals(fileId1, record._2.get().getRight());
       } else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
         assertTrue(record._2.isPresent());
-        if (record._1.getPartitionPath().equals("2015/01/31")) {
+        if (record._1.getPartitionPath().equals(partition2)) {
           assertEquals(fileId3, record._2.get().getRight());
         } else {
           assertEquals(fileId2, record._2.get().getRight());
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 9d25907b4bf9d..3ad8952feea84 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
@@ -18,22 +18,25 @@
 
 package org.apache.hudi.index.bloom;
 
+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.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.testutils.RawTripTestPayload;
 import org.apache.hudi.common.util.collection.Pair;
 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;
-import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
 
 import org.apache.avro.Schema;
+import org.apache.hadoop.fs.Path;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
 import org.junit.jupiter.api.AfterEach;
@@ -41,12 +44,14 @@
 import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 import java.util.stream.Collectors;
 
 import scala.Tuple2;
@@ -59,7 +64,7 @@
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
-public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
+public class TestHoodieGlobalBloomIndex extends TestHoodieMetadataBase {
 
   private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.avsc", true);
 
@@ -67,7 +72,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
   public void setUp() throws Exception {
     initSparkContexts();
     initPath();
+    initFileSystem();
     initMetaClient();
+    HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM);
+    HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withIndexConfig(indexBuilder.build())
+        .build();
+    writeClient = getHoodieWriteClient(config);
   }
 
   @AfterEach
@@ -81,13 +92,15 @@ public void testLoadInvolvedFiles() throws Exception {
     HoodieGlobalBloomIndex index =
         new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
 
     // Create some partitions, and put some files, along with the meta file
     // "2016/01/21": 0 file
     // "2016/04/01": 1 file (2_0_20160401010101.parquet)
     // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
-    testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
+    final String p1 = "2016/01/21";
+    final String p2 = "2016/04/01";
+    final String p3 = "2015/03/12";
 
     RawTripTestPayload rowChange1 =
         new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
@@ -107,16 +120,46 @@ public void testLoadInvolvedFiles() throws Exception {
         new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
 
     // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
-    List partitions = Arrays.asList("2016/01/21", "2016/04/01");
+    List partitions = Arrays.asList(p1, p2);
     // partitions will NOT be respected by this loadInvolvedFiles(...) call
     List> filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
     // Still 0, as no valid commit
     assertEquals(0, filesList.size());
 
-    testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
-    testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
-        .withInserts("2015/03/12", "3", record1)
-        .withInserts("2015/03/12", "4", record2, record3, record4);
+    final String fileId1 = "1";
+    final String fileId2 = "2";
+    final String fileId3 = "3";
+    final String fileId4 = "4";
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
+
+    final String c1 = "20160401010101";
+    Path baseFilePath = testTable.forCommit(c1).withInserts(p2, fileId2, Collections.emptyList());
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p2,
+        k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(c1, WriteOperationType.UPSERT, Collections.singletonList(p2),
+        partitionToFilesNameLengthMap, false, false);
+
+    final String c2 = "20150312101010";
+    testTable.forCommit(c2);
+    baseFilePath = testTable.withInserts(p3, fileId1, Collections.emptyList());
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(p3,
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+
+    baseFilePath = testTable.withInserts(p3, fileId3, Collections.singletonList(record1));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p3,
+        k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+
+    baseFilePath = testTable.withInserts(p3, fileId4, Arrays.asList(record2, record3, record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p3,
+        k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
+
+    testTable.doWriteOperation(c2, WriteOperationType.UPSERT, Collections.singletonList(p3),
+        partitionToFilesNameLengthMap, false, false);
 
     filesList = index.loadColumnRangesFromFiles(partitions, context, hoodieTable);
     assertEquals(4, filesList.size());
@@ -185,17 +228,21 @@ public void testExplodeRecordRDDWithFileComparisons() {
   @Test
   public void testTagLocation() throws Exception {
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
-        .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(false).build()).build();
-    HoodieGlobalBloomIndex index =
-        new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
+        .withIndexConfig(HoodieIndexConfig.newBuilder()
+            .withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM)
+            .withBloomIndexUpdatePartitionPath(false)
+            .build())
+        .build();
+    HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
 
     // Create some partitions, and put some files, along with the meta file
     // "2016/01/21": 0 file
     // "2016/04/01": 1 file (2_0_20160401010101.parquet)
     // "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
-    testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
+    final String partition2 = "2016/04/01";
+    final String partition3 = "2015/03/12";
 
     RawTripTestPayload rowChange1 =
         new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
@@ -223,13 +270,49 @@ public void testTagLocation() throws Exception {
     HoodieRecord record5 =
         new HoodieAvroRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5);
 
-    JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
+    final String fileId1 = UUID.randomUUID().toString();
+    final String fileId2 = UUID.randomUUID().toString();
+    final String fileId3 = UUID.randomUUID().toString();
+    final String fileId4 = UUID.randomUUID().toString();
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
 
     // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
-    String fileId1 = testTable.addCommit("1000").getFileIdWithInserts("2016/04/01", record1);
-    String fileId2 = testTable.addCommit("2000").getFileIdWithInserts("2015/03/12");
-    String fileId3 = testTable.addCommit("3000").getFileIdWithInserts("2015/03/12", record2);
-    String fileId4 = testTable.addCommit("4000").getFileIdWithInserts("2015/03/12", record4);
+    String commitTime = "0000001";
+    Path baseFilePath = testTable.forCommit(commitTime).withInserts(partition2, fileId1, Collections.singletonList(record1));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition2,
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition2),
+        partitionToFilesNameLengthMap, false, false);
+
+    commitTime = "0000002";
+    baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId2, Collections.emptyList());
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition3,
+        k -> new ArrayList<>()).add(Pair.of(fileId2, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
+        partitionToFilesNameLengthMap, false, false);
+
+    commitTime = "0000003";
+    baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId3, Collections.singletonList(record2));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition3,
+        k -> new ArrayList<>()).add(Pair.of(fileId3, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
+        partitionToFilesNameLengthMap, false, false);
+
+    commitTime = "0000004";
+    baseFilePath = testTable.forCommit(commitTime).withInserts(partition3, fileId4, Collections.singletonList(record4));
+    baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.clear();
+    partitionToFilesNameLengthMap.computeIfAbsent(partition3,
+        k -> new ArrayList<>()).add(Pair.of(fileId4, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Collections.singletonList(partition3),
+        partitionToFilesNameLengthMap, false, false);
+
+    JavaRDD recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
 
     // partitions will NOT be respected by this loadInvolvedFiles(...) call
     JavaRDD taggedRecordRDD = tagLocation(index, recordRDD, hoodieTable);
@@ -266,12 +349,15 @@ public void testTagLocation() throws Exception {
   public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
         .withPath(basePath)
-        .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
+        .withIndexConfig(HoodieIndexConfig.newBuilder()
+            .withIndexType(HoodieIndex.IndexType.GLOBAL_BLOOM)
+            .withBloomIndexUpdatePartitionPath(true)
+            .build())
         .build();
     HoodieGlobalBloomIndex index =
         new HoodieGlobalBloomIndex(config, SparkHoodieBloomIndexHelper.getInstance());
     HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
-    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(hoodieTable, SCHEMA);
+    HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(metaClient, SCHEMA, metadataWriter);
     final String p1 = "2016/01/31";
     final String p2 = "2016/02/28";
 
@@ -309,7 +395,16 @@ public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
             new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
             incomingPayloadSamePartition);
 
-    testTable.addCommit("1000").getFileIdWithInserts(p1, originalRecord);
+    final String fileId1 = UUID.randomUUID().toString();
+    final Map>> partitionToFilesNameLengthMap = new HashMap<>();
+
+    final String commitTime = "0000001";
+    Path baseFilePath = testTable.forCommit(commitTime).withInserts(p1, fileId1, Collections.singletonList(originalRecord));
+    long baseFileLength = fs.getFileStatus(baseFilePath).getLen();
+    partitionToFilesNameLengthMap.computeIfAbsent(p1,
+        k -> new ArrayList<>()).add(Pair.of(fileId1, Integer.valueOf((int) baseFileLength)));
+    testTable.doWriteOperation(commitTime, WriteOperationType.UPSERT, Arrays.asList(p1),
+        partitionToFilesNameLengthMap, false, false);
 
     // test against incoming record with a different partition
     JavaRDD recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
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 f51a169dd9b44..552e85af4c66b 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
@@ -1077,6 +1077,8 @@ private Pair genera
       writeStat.setPartitionPath(partition);
       writeStat.setPath(partition + "/" + getBaseFilename(instantTime, newFileId));
       writeStat.setFileId(newFileId);
+      writeStat.setTotalWriteBytes(1);
+      writeStat.setFileSizeInBytes(1);
       replaceMetadata.addWriteStat(partition, writeStat);
     }
     return Pair.of(requestedReplaceMetadata, replaceMetadata);
@@ -1756,6 +1758,8 @@ protected static HoodieCommitMetadata generateCommitMetadata(
       writeStat.setPartitionPath(partitionPath);
       writeStat.setPath(partitionPath + "/" + getBaseFilename(instantTime, f));
       writeStat.setFileId(f);
+      writeStat.setTotalWriteBytes(1);
+      writeStat.setFileSizeInBytes(1);
       metadata.addWriteStat(partitionPath, writeStat);
     }));
     return metadata;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
index 2955147b4053f..73b1da95648e2 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java
@@ -35,6 +35,7 @@
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
 import org.apache.hudi.common.table.view.TableFileSystemView;
+import org.apache.hudi.common.testutils.FileCreateUtils;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
@@ -180,6 +181,61 @@ public void testInlineScheduleCompaction(boolean scheduleInlineCompaction) throw
     }
   }
 
+  @Test
+  public void testRepeatedRollbackOfCompaction() throws Exception {
+    boolean scheduleInlineCompaction = false;
+    HoodieFileFormat fileFormat = HoodieFileFormat.PARQUET;
+    Properties properties = new Properties();
+    properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), fileFormat.toString());
+    HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties);
+
+    HoodieWriteConfig cfg = getConfigBuilder(false)
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
+            .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(2).withPreserveCommitMetadata(true).withScheduleInlineCompaction(scheduleInlineCompaction).build())
+        .build();
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
+
+      HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
+      /*
+       * Write 1 (only inserts)
+       */
+      String newCommitTime = "001";
+      client.startCommitWithTime(newCommitTime);
+
+      List records = dataGen.generateInserts(newCommitTime, 200);
+      Stream dataFiles = insertRecordsToMORTable(metaClient, records, client, cfg, newCommitTime, true);
+      assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit");
+
+      /*
+       * Write 2 (updates)
+       */
+      newCommitTime = "004";
+      client.startCommitWithTime(newCommitTime);
+      records = dataGen.generateUpdates(newCommitTime, 100);
+      updateRecordsInMORTable(metaClient, records, client, cfg, newCommitTime, true);
+
+      Option compactionInstant = client.scheduleCompaction(Option.empty());
+      client.compact(compactionInstant.get());
+
+      // trigger compaction again.
+      client.compact(compactionInstant.get());
+
+      metaClient.reloadActiveTimeline();
+      // verify that there is no new rollback instant generated
+      HoodieInstant rollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get();
+      FileCreateUtils.deleteRollbackCommit(metaClient.getBasePath().substring(metaClient.getBasePath().indexOf(":") + 1),
+          rollbackInstant.getTimestamp());
+      metaClient.reloadActiveTimeline();
+      SparkRDDWriteClient client1 = getHoodieWriteClient(cfg);
+      // trigger compaction again.
+      client1.compact(compactionInstant.get());
+      metaClient.reloadActiveTimeline();
+      // verify that there is no new rollback instant generated
+      HoodieInstant newRollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get();
+      assertEquals(rollbackInstant.getTimestamp(), newRollbackInstant.getTimestamp());
+    }
+  }
+
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
   public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws Exception {
@@ -264,7 +320,7 @@ public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws E
           .map(baseFile -> new Path(baseFile.getPath()).getParent().toString())
           .collect(Collectors.toList());
       List recordsRead =
-          HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf(), inputPaths, basePath(), new JobConf(hadoopConf()), true, false);
+          HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf(), inputPaths, basePath(), new JobConf(hadoopConf()), true, populateMetaFields);
       // Wrote 20 records and deleted 20 records, so remaining 20-20 = 0
       assertEquals(0, recordsRead.size(), "Must contain 0 records");
     }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
index d552955030baa..7655cf93f104b 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
@@ -21,7 +21,6 @@
 
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -150,11 +149,7 @@ void testCOWToMORConvertedTableRollback(boolean rollbackUsingMarkers) throws Exc
   void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) throws Exception {
     // NOTE: First writer will have Metadata table DISABLED
     HoodieWriteConfig.Builder cfgBuilder =
-        getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE)
-            .withMetadataConfig(
-                HoodieMetadataConfig.newBuilder()
-                    .enable(false)
-                    .build());
+        getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE);
     
     addConfigsForPopulateMetaFields(cfgBuilder, true);
     HoodieWriteConfig cfg = cfgBuilder.build();
@@ -209,7 +204,7 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro
       final String commitTime1 = "002";
       // WriteClient with custom config (disable small file handling)
       // NOTE: Second writer will have Metadata table ENABLED
-      try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(false));) {
+      try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(true));) {
         secondClient.startCommitWithTime(commitTime1);
 
         List copyOfRecords = new ArrayList<>(records);
@@ -245,8 +240,8 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro
       /*
        * Write 3 (inserts + updates - testing successful delta commit)
        */
-      final String commitTime2 = "002";
-      try (SparkRDDWriteClient thirdClient = getHoodieWriteClient(cfg);) {
+      final String commitTime2 = "003";
+      try (SparkRDDWriteClient thirdClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(true));) {
         thirdClient.startCommitWithTime(commitTime2);
 
         List copyOfRecords = new ArrayList<>(records);
@@ -287,7 +282,7 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro
         /*
          * Write 4 (updates)
          */
-        newCommitTime = "003";
+        newCommitTime = "004";
         thirdClient.startCommitWithTime(newCommitTime);
 
         writeStatusJavaRDD = thirdClient.upsert(writeRecords, newCommitTime);
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 fd2af1cdca25a..67623709524ff 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
@@ -80,6 +80,20 @@ public void tearDown() throws Exception {
     cleanupResources();
   }
 
+  @Test
+  public void testMarkerBasedRollbackAppend() throws Exception {
+    HoodieTestTable testTable = HoodieTestTable.of(metaClient);
+    String f0 = testTable.addRequestedCommit("000")
+        .getFileIdsWithBaseFilesInPartitions("partA").get("partA");
+    testTable.forCommit("001")
+        .withMarkerFile("partA", f0, IOType.APPEND);
+
+    HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient);
+    List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(),
+        "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"));
+    assertEquals(1, rollbackRequests.size());
+  }
+
   @Test
   public void testCopyOnWriteRollbackWithTestTable() throws Exception {
     // given: wrote some base files and corresponding markers
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
index f339f5ed910db..71e4b4b4e6e3f 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
@@ -102,6 +102,7 @@
 import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertLinesMatch;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
@@ -617,21 +618,7 @@ protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableM
     Collections.sort(fsFileNames);
     Collections.sort(metadataFilenames);
 
-    if ((fsFileNames.size() != metadataFilenames.size()) || (!fsFileNames.equals(metadataFilenames))) {
-      LOG.info("*** File system listing = " + Arrays.toString(fsFileNames.toArray()));
-      LOG.info("*** Metadata listing = " + Arrays.toString(metadataFilenames.toArray()));
-
-      for (String fileName : fsFileNames) {
-        if (!metadataFilenames.contains(fileName)) {
-          LOG.error(partition + "FsFilename " + fileName + " not found in Meta data");
-        }
-      }
-      for (String fileName : metadataFilenames) {
-        if (!fsFileNames.contains(fileName)) {
-          LOG.error(partition + "Metadata file " + fileName + " not found in original FS");
-        }
-      }
-    }
+    assertLinesMatch(fsFileNames, metadataFilenames);
     assertEquals(fsStatuses.length, partitionToFilesMap.get(partitionPath.toString()).length);
 
     // Block sizes should be valid
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java
index ca7bb4e0117c5..8940223926089 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkWriteableTestTable.java
@@ -30,6 +30,7 @@
 
 import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -65,7 +66,7 @@ public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient,
   public static HoodieSparkWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema,
                                                  HoodieTableMetadataWriter metadataWriter) {
     BloomFilter filter = BloomFilterFactory
-        .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
+        .createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.DYNAMIC_V0.name());
     return of(metaClient, schema, filter, metadataWriter);
   }
 
@@ -108,11 +109,11 @@ public HoodieSparkWriteableTestTable withInserts(String partition, String fileId
   }
 
   public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception {
-    return withInserts(partition, fileId, Arrays.asList(records));
+    withInserts(partition, fileId, Arrays.asList(records));
+    return this;
   }
 
-  public HoodieSparkWriteableTestTable withInserts(String partition, String fileId, List records) throws Exception {
-    super.withInserts(partition, fileId, records, new SparkTaskContextSupplier());
-    return this;
+  public Path withInserts(String partition, String fileId, List records) throws Exception {
+    return super.withInserts(partition, fileId, records, new SparkTaskContextSupplier());
   }
 }
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 94e080cae4804..f9676c6c477be 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
@@ -19,6 +19,13 @@
 
 package org.apache.hudi.testutils;
 
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.AvroConversionUtils;
 import org.apache.hudi.client.HoodieReadClient;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
@@ -28,6 +35,7 @@
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -42,6 +50,7 @@
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.keygen.SimpleKeyGenerator;
 import org.apache.hudi.table.HoodieSparkTable;
@@ -50,14 +59,11 @@
 import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
 import org.apache.hudi.testutils.providers.SparkProvider;
 import org.apache.hudi.timeline.service.TimelineService;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
 import org.apache.spark.sql.SparkSession;
 import org.junit.jupiter.api.AfterAll;
@@ -69,6 +75,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
@@ -348,6 +355,21 @@ protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, Boolean
         .withRollbackUsingMarkers(rollbackUsingMarkers);
   }
 
+  protected Dataset toDataset(List records, Schema schema) {
+    List avroRecords = records.stream()
+        .map(r -> {
+          HoodieRecordPayload payload = (HoodieRecordPayload) r.getData();
+          try {
+            return (GenericRecord) payload.getInsertValue(schema).get();
+          } catch (IOException e) {
+            throw new HoodieIOException("Failed to extract Avro payload", e);
+          }
+        })
+        .collect(Collectors.toList());
+    JavaRDD jrdd = jsc.parallelize(avroRecords, 2);
+    return AvroConversionUtils.createDataFrame(jrdd.rdd(), schema.toString(), spark);
+  }
+
   protected int incrementTimelineServicePortToUse() {
     // Increment the timeline service port for each individual test
     // to avoid port reuse causing failures
diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
index 428da925c49ea..9eae74e928c25 100644
--- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
+++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java
@@ -18,8 +18,6 @@
 
 package org.apache.hudi;
 
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -37,6 +35,9 @@
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieIOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -333,7 +334,7 @@ private static long fileSliceSize(FileSlice fileSlice) {
     return fileSlice.getBaseFile().map(BaseFile::getFileLen).orElse(0L) + logFileSize;
   }
 
-  protected static final class PartitionPath {
+  public static final class PartitionPath {
     final String path;
     final Object[] values;
 
@@ -342,6 +343,10 @@ public PartitionPath(String path, Object[] values) {
       this.values = values;
     }
 
+    public String getPath() {
+      return path;
+    }
+
     Path fullPartitionPath(String basePath) {
       if (!path.isEmpty()) {
         return new Path(basePath, path);
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 209721e24a8d9..5cb18dc8d1509 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
@@ -176,7 +176,7 @@ public static Schema addMetadataFields(Schema schema) {
   /**
    * Adds the Hoodie metadata fields to the given schema.
    *
-   * @param schema The schema
+   * @param schema             The schema
    * @param withOperationField Whether to include the '_hoodie_operation' field
    */
   public static Schema addMetadataFields(Schema schema, boolean withOperationField) {
@@ -276,7 +276,7 @@ public static Schema getSchemaForFields(Schema fileSchema, List fields)
     List toBeAddedFields = new ArrayList<>();
     Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false);
 
-    for (Schema.Field schemaField: fileSchema.getFields()) {
+    for (Schema.Field schemaField : fileSchema.getFields()) {
       if (fields.contains(schemaField.name())) {
         toBeAddedFields.add(new Schema.Field(schemaField.name(), schemaField.schema(), schemaField.doc(), schemaField.defaultVal()));
       }
@@ -303,7 +303,7 @@ public static GenericRecord addOperationToRecord(GenericRecord record, HoodieOpe
    * engines have varying constraints regarding treating the case-sensitivity of fields, its best to let caller
    * determine that.
    *
-   * @param schema Passed in schema
+   * @param schema        Passed in schema
    * @param newFieldNames Null Field names to be added
    */
   public static Schema appendNullSchemaFields(Schema schema, List newFieldNames) {
@@ -382,10 +382,34 @@ public static GenericRecord rewriteRecord(GenericRecord oldRecord, Schema newSch
     return newRecord;
   }
 
+  public static GenericRecord rewriteRecord(GenericRecord genericRecord, Schema newSchema, boolean copyOverMetaFields, GenericRecord fallbackRecord) {
+    GenericRecord newRecord = new GenericData.Record(newSchema);
+    boolean isSpecificRecord = genericRecord instanceof SpecificRecordBase;
+    for (Schema.Field f : newSchema.getFields()) {
+      if (!(isSpecificRecord && isMetadataField(f.name()))) {
+        copyOldValueOrSetDefault(genericRecord, newRecord, f);
+      }
+      if (isMetadataField(f.name()) && copyOverMetaFields) {
+        // if meta field exists in primary generic record, copy over.
+        if (genericRecord.getSchema().getField(f.name()) != null) {
+          copyOldValueOrSetDefault(genericRecord, newRecord, f);
+        } else if (fallbackRecord != null && fallbackRecord.getSchema().getField(f.name()) != null) {
+          // if not, try to copy from the fallback record.
+          copyOldValueOrSetDefault(fallbackRecord, newRecord, f);
+        }
+      }
+    }
+    if (!GenericData.get().validate(newSchema, newRecord)) {
+      throw new SchemaCompatibilityException(
+          "Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema);
+    }
+    return newRecord;
+  }
+
   /**
    * Converts list of {@link GenericRecord} provided into the {@link GenericRecord} adhering to the
    * provided {@code newSchema}.
-   *
+   * 

* To better understand conversion rules please check {@link #rewriteRecord(GenericRecord, Schema)} */ public static List rewriteRecords(List records, Schema newSchema) { @@ -491,9 +515,8 @@ public static Object getNestedFieldVal(GenericRecord record, String fieldName, b * Returns the string value of the given record {@code rec} and field {@code fieldName}. * The field and value both could be missing. * - * @param rec The record + * @param rec 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 */ @@ -507,7 +530,7 @@ public static Option getNullableValAsString(GenericRecord rec, String fi * This method converts values for fields with certain Avro/Parquet data types that require special handling. * * @param fieldSchema avro field schema - * @param fieldValue avro field value + * @param fieldValue avro field value * @return field value either converted (for certain data types) or as it is. */ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object fieldValue, boolean consistentLogicalTimestampEnabled) { @@ -527,15 +550,15 @@ public static Object convertValueForSpecificDataTypes(Schema fieldSchema, Object /** * This method converts values for fields with certain Avro Logical data types that require special handling. - * + *

* Logical Date Type is converted to actual Date value instead of Epoch Integer which is how it is * represented/stored in parquet. - * + *

* Decimal Data Type is converted to actual decimal value instead of bytes/fixed which is how it is * represented/stored in parquet. * * @param fieldSchema avro field schema - * @param fieldValue avro field value + * @param fieldValue avro field value * @return field value either converted (for certain data types) or as it is. */ private static Object convertValueForAvroLogicalTypes(Schema fieldSchema, Object fieldValue, boolean consistentLogicalTimestampEnabled) { @@ -569,6 +592,7 @@ public static Schema getNullSchema() { /** * Sanitizes Name according to Avro rule for names. * Removes characters other than the ones mentioned in https://avro.apache.org/docs/current/spec.html#names . + * * @param name input name * @return sanitized name */ 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 86ff64177b73a..7d964f3582d69 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 @@ -169,6 +169,12 @@ public final class HoodieMetadataConfig extends HoodieConfig { + "store the column ranges and will be used for pruning files during the index lookups. " + "Only applies if " + ENABLE_METADATA_INDEX_COLUMN_STATS.key() + " is enabled."); + public static final ConfigProperty COLUMN_STATS_INDEX_PARALLELISM = ConfigProperty + .key(METADATA_PREFIX + ".index.column.stats.parallelism") + .defaultValue(10) + .sinceVersion("0.11.0") + .withDocumentation("Parallelism to use, when generating column stats index."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") .defaultValue(false) @@ -223,6 +229,10 @@ public int getColumnStatsIndexFileGroupCount() { return getIntOrDefault(METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT); } + public int getColumnStatsIndexParallelism() { + return getIntOrDefault(COLUMN_STATS_INDEX_PARALLELISM); + } + public boolean enableMetrics() { return getBoolean(METRICS_ENABLE); } @@ -285,6 +295,11 @@ public Builder withMetadataIndexColumnStatsFileGroupCount(int fileGroupCount) { return this; } + public Builder withColumnStatsIndexParallelism(int parallelism) { + metadataConfig.setValue(COLUMN_STATS_INDEX_PARALLELISM, String.valueOf(parallelism)); + return this; + } + public Builder withMetadataIndexForAllColumns(boolean enable) { metadataConfig.setValue(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS, String.valueOf(enable)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/OrderedProperties.java b/hudi-common/src/main/java/org/apache/hudi/common/config/OrderedProperties.java new file mode 100644 index 0000000000000..fa2a61574a84e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/OrderedProperties.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.config; + +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; + +/** + * An extension of {@link java.util.Properties} that maintains the order. + * The implementation is not thread-safe. + */ +public class OrderedProperties extends Properties { + + private final HashSet keys = new LinkedHashSet<>(); + + public OrderedProperties() { + super(null); + } + + public OrderedProperties(Properties defaults) { + if (Objects.nonNull(defaults)) { + for (String key : defaults.stringPropertyNames()) { + put(key, defaults.getProperty(key)); + } + } + } + + @Override + public Enumeration propertyNames() { + return Collections.enumeration(keys); + } + + @Override + public synchronized Enumeration keys() { + return Collections.enumeration(keys); + } + + @Override + public Set stringPropertyNames() { + Set set = new LinkedHashSet<>(); + for (Object key : this.keys) { + if (key instanceof String) { + set.add((String) key); + } + } + return set; + } + + public synchronized void putAll(Properties t) { + for (Map.Entry e : t.entrySet()) { + if (!containsKey(String.valueOf(e.getKey()))) { + keys.add(e.getKey()); + } + super.put(e.getKey(), e.getValue()); + } + } + + @Override + public synchronized Object put(Object key, Object value) { + keys.remove(key); + keys.add(key); + return super.put(key, value); + } + + public synchronized Object putIfAbsent(Object key, Object value) { + if (!containsKey(String.valueOf(key))) { + keys.add(key); + } + return super.putIfAbsent(key, value); + } + + @Override + public Object remove(Object key) { + keys.remove(key); + return super.remove(key); + } +} 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 6639e88d56f3f..2688e6454d814 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 @@ -20,15 +20,9 @@ import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; -import java.util.Enumeration; -import java.util.HashSet; -import java.util.LinkedHashSet; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Properties; -import java.util.Set; import java.util.stream.Collectors; /** @@ -36,8 +30,6 @@ */ public class TypedProperties extends Properties implements Serializable { - private final HashSet keys = new LinkedHashSet<>(); - public TypedProperties() { super(null); } @@ -50,56 +42,6 @@ public TypedProperties(Properties defaults) { } } - @Override - public Enumeration propertyNames() { - return Collections.enumeration(keys); - } - - @Override - public synchronized Enumeration keys() { - return Collections.enumeration(keys); - } - - @Override - public Set stringPropertyNames() { - Set set = new LinkedHashSet<>(); - for (Object key : this.keys) { - if (key instanceof String) { - set.add((String) key); - } - } - return set; - } - - public synchronized void putAll(Properties t) { - for (Map.Entry e : t.entrySet()) { - if (!containsKey(String.valueOf(e.getKey()))) { - keys.add(e.getKey()); - } - super.put(e.getKey(), e.getValue()); - } - } - - @Override - public synchronized Object put(Object key, Object value) { - keys.remove(key); - keys.add(key); - return super.put(key, value); - } - - public synchronized Object putIfAbsent(Object key, Object value) { - if (!containsKey(String.valueOf(key))) { - keys.add(key); - } - return super.putIfAbsent(key, value); - } - - @Override - public Object remove(Object key) { - keys.remove(key); - return super.remove(key); - } - private void checkKey(String property) { if (!containsKey(property)) { throw new IllegalArgumentException("Property " + property + " not found"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java index 5e4b445dfc85e..5a588eafa5f3f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java @@ -89,10 +89,15 @@ private static Option updateEventTime(GenericRecord record, Properties p boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(properties.getProperty( KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())); + String eventTimeField = properties + .getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY); + if (eventTimeField == null) { + return Option.empty(); + } return Option.ofNullable( HoodieAvroUtils.getNestedFieldVal( record, - properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), + eventTimeField, true, consistentLogicalTimestampEnabled) ); @@ -118,14 +123,18 @@ protected boolean needUpdatingPersistedRecord(IndexedRecord currentValue, * NOTE: Deletes sent via EmptyHoodieRecordPayload and/or Delete operation type do not hit this code path * and need to be dealt with separately. */ + String orderField = properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY); + if (orderField == null) { + return true; + } boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(properties.getProperty( KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())); Object persistedOrderingVal = HoodieAvroUtils.getNestedFieldVal((GenericRecord) currentValue, - properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), + orderField, true, consistentLogicalTimestampEnabled); Comparable incomingOrderingVal = (Comparable) HoodieAvroUtils.getNestedFieldVal((GenericRecord) incomingRecord, - properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), + orderField, true, consistentLogicalTimestampEnabled); return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java index acf5b2298987a..d098c4ff7c2b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieColumnRangeMetadata.java @@ -18,12 +18,16 @@ package org.apache.hudi.common.model; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; import java.util.Objects; +import java.util.function.BiFunction; /** * Hoodie Range metadata. */ -public class HoodieColumnRangeMetadata { +public class HoodieColumnRangeMetadata implements Serializable { private final String filePath; private final String columnName; private final T minValue; @@ -33,6 +37,20 @@ public class HoodieColumnRangeMetadata { private final long totalSize; private final long totalUncompressedSize; + public static final BiFunction, HoodieColumnRangeMetadata, HoodieColumnRangeMetadata> COLUMN_RANGE_MERGE_FUNCTION = + (oldColumnRange, newColumnRange) -> new HoodieColumnRangeMetadata<>( + newColumnRange.getFilePath(), + newColumnRange.getColumnName(), + (Comparable) Arrays.asList(oldColumnRange.getMinValue(), newColumnRange.getMinValue()) + .stream().filter(Objects::nonNull).min(Comparator.naturalOrder()).orElse(null), + (Comparable) Arrays.asList(oldColumnRange.getMinValue(), newColumnRange.getMinValue()) + .stream().filter(Objects::nonNull).max(Comparator.naturalOrder()).orElse(null), + oldColumnRange.getNullCount() + newColumnRange.getNullCount(), + oldColumnRange.getValueCount() + newColumnRange.getValueCount(), + oldColumnRange.getTotalSize() + newColumnRange.getTotalSize(), + oldColumnRange.getTotalUncompressedSize() + newColumnRange.getTotalUncompressedSize() + ); + public HoodieColumnRangeMetadata(final String filePath, final String columnName, final T minValue, final T maxValue, final long nullCount, long valueCount, long totalSize, long totalUncompressedSize) { this.filePath = filePath; @@ -114,4 +132,18 @@ public String toString() { + ", totalUncompressedSize=" + totalUncompressedSize + '}'; } + + /** + * Statistics that is collected in {@link org.apache.hudi.metadata.MetadataPartitionType#COLUMN_STATS} index. + */ + public static final class Stats { + public static final String VALUE_COUNT = "value_count"; + public static final String NULL_COUNT = "null_count"; + public static final String MIN = "min"; + public static final String MAX = "max"; + public static final String TOTAL_SIZE = "total_size"; + public static final String TOTAL_UNCOMPRESSED_SIZE = "total_uncompressed_size"; + + private Stats() { } + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java index c97743f4d115e..cf3bb52263366 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDeltaWriteStat.java @@ -19,9 +19,12 @@ package org.apache.hudi.common.model; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.hudi.common.util.Option; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import java.util.Map; /** * Statistics about a single Hoodie delta log operation. @@ -33,6 +36,7 @@ public class HoodieDeltaWriteStat extends HoodieWriteStat { private long logOffset; private String baseFile; private List logFiles = new ArrayList<>(); + private Option> recordsStats = Option.empty(); public void setLogVersion(int logVersion) { this.logVersion = logVersion; @@ -69,4 +73,24 @@ public void addLogFiles(String logFile) { public List getLogFiles() { return logFiles; } + + public void setRecordsStats(RecordsStats stats) { + recordsStats = Option.of(stats); + } + + public Option> getRecordsStats() { + return recordsStats; + } + + public static class RecordsStats implements Serializable { + private final T recordsStats; + + public RecordsStats(T recordsStats) { + this.recordsStats = recordsStats; + } + + public T getStats() { + return recordsStats; + } + } } 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 ac30766dd2f03..0f21ae1bef185 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java @@ -42,6 +42,8 @@ public abstract class HoodieRecord implements Serializable { public static final String OPERATION_METADATA_FIELD = "_hoodie_operation"; public static final String HOODIE_IS_DELETED = "_hoodie_is_deleted"; + public static int FILENAME_METADATA_FIELD_POS = 4; + public static final List HOODIE_META_COLUMNS = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD, RECORD_KEY_METADATA_FIELD, PARTITION_PATH_METADATA_FIELD, FILENAME_METADATA_FIELD); 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 dc010366cd3b5..077cc81ef1a60 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 @@ -18,19 +18,13 @@ package org.apache.hudi.common.table; -import org.apache.avro.Schema; - -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - 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; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieRecord; @@ -47,12 +41,17 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; +import java.time.Instant; import java.util.Arrays; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.Properties; @@ -172,9 +171,9 @@ public class HoodieTableConfig extends HoodieConfig { .noDefaultValue() .withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table"); - public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key("hoodie.populate.meta.fields") - .defaultValue("true") + .defaultValue(true) .withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated " + "and incremental queries will not be functional. This is only meant to be used for append only/immutable data for batch processing"); @@ -220,10 +219,7 @@ public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName setValue(PAYLOAD_CLASS_NAME, payloadClassName); // FIXME(vc): wonder if this can be removed. Need to look into history. try (FSDataOutputStream outputStream = fs.create(propertyPath)) { - if (!isValidChecksum()) { - setValue(TABLE_CHECKSUM, String.valueOf(generateChecksum(props))); - } - props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); + storeProperties(props, outputStream); } } } catch (IOException e) { @@ -233,6 +229,34 @@ public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName "hoodie.properties file seems invalid. Please check for left over `.updated` files if any, manually copy it to hoodie.properties and retry"); } + private static Properties getOrderedPropertiesWithTableChecksum(Properties props) { + Properties orderedProps = new OrderedProperties(props); + orderedProps.put(TABLE_CHECKSUM.key(), String.valueOf(generateChecksum(props))); + return orderedProps; + } + + /** + * Write the properties to the given output stream and return the table checksum. + * + * @param props - properties to be written + * @param outputStream - output stream to which properties will be written + * @return return the table checksum + * @throws IOException + */ + private static String storeProperties(Properties props, FSDataOutputStream outputStream) throws IOException { + String checksum; + if (props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props)) { + checksum = props.getProperty(TABLE_CHECKSUM.key()); + props.store(outputStream, "Updated at " + Instant.now()); + } else { + Properties propsWithChecksum = getOrderedPropertiesWithTableChecksum(props); + propsWithChecksum.store(outputStream, "Properties saved on " + Instant.now()); + checksum = propsWithChecksum.getProperty(TABLE_CHECKSUM.key()); + props.setProperty(TABLE_CHECKSUM.key(), checksum); + } + return checksum; + } + private boolean isValidChecksum() { return contains(TABLE_CHECKSUM) && validateChecksum(props); } @@ -316,13 +340,7 @@ private static void modify(FileSystem fs, Path metadataFolder, Properties modify Properties props = new TypedProperties(); props.load(in); modifyFn.accept(props, modifyProps); - if (props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props)) { - checksum = props.getProperty(TABLE_CHECKSUM.key()); - } else { - checksum = String.valueOf(generateChecksum(props)); - props.setProperty(TABLE_CHECKSUM.key(), checksum); - } - props.store(out, "Updated at " + System.currentTimeMillis()); + checksum = storeProperties(props, out); } // 4. verify and remove backup. try (FSDataInputStream in = fs.open(cfgPath)) { @@ -385,12 +403,7 @@ public static void create(FileSystem fs, Path metadataFolder, Properties propert if (hoodieConfig.contains(TIMELINE_TIMEZONE)) { HoodieInstantTimeGenerator.setCommitTimeZone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getString(TIMELINE_TIMEZONE))); } - if (hoodieConfig.contains(TABLE_CHECKSUM)) { - hoodieConfig.setValue(TABLE_CHECKSUM, hoodieConfig.getString(TABLE_CHECKSUM)); - } else { - hoodieConfig.setValue(TABLE_CHECKSUM, String.valueOf(generateChecksum(hoodieConfig.getProps()))); - } - hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); + storeProperties(hoodieConfig.getProps(), outputStream); } } 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 a84a9482a6707..f2db4d692866a 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 @@ -143,7 +143,7 @@ public Schema getTableAvroSchemaFromDataFile() { * @throws Exception */ public Schema getTableAvroSchema() throws Exception { - return getTableAvroSchema(metaClient.getTableConfig().populateMetaFields()); + return getTableAvroSchema(true); } /** 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 d0ab73ab01552..882e1057c8043 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 @@ -59,18 +59,15 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader implements Iterable> { private static final Logger LOG = LogManager.getLogger(HoodieMergedLogRecordScanner.class); - + // A timer for calculating elapsed time in millis + public final HoodieTimer timer = new HoodieTimer(); // Final map of compacted/merged records protected final ExternalSpillableMap> records; - // count of merged records in log private long numMergedRecordsInLog; private long maxMemorySizeInBytes; - // Stores the total time taken to perform reading and merging of log blocks private long totalTimeTakenToReadAndMergeBlocks; - // A timer for calculating elapsed time in millis - public final HoodieTimer timer = new HoodieTimer(); @SuppressWarnings("unchecked") protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema, @@ -143,9 +140,11 @@ protected void processNextRecord(HoodieRecord hoo HoodieRecord oldRecord = records.get(key); HoodieRecordPayload oldValue = oldRecord.getData(); HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(oldValue); - boolean choosePrev = combinedValue.equals(oldValue); - HoodieOperation operation = choosePrev ? oldRecord.getOperation() : hoodieRecord.getOperation(); - records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation)); + // If combinedValue is oldValue, no need rePut oldRecord + if (combinedValue != oldValue) { + HoodieOperation operation = hoodieRecord.getOperation(); + records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation)); + } } else { // Put the record as is records.put(key, hoodieRecord); @@ -187,11 +186,11 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder { protected boolean isBitCaskDiskMapCompressionEnabled = HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue(); // incremental filtering protected Option instantRange = Option.empty(); + protected String partitionName; // auto scan default true private boolean autoScan = true; // operation field default false private boolean withOperationField = false; - protected String partitionName; @Override public Builder withFileSystem(FileSystem fs) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index 9c0e33b9527b7..33b847b4c62c5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -69,7 +69,7 @@ public static String createNewInstantTime(long milliseconds) { return lastInstantTime.updateAndGet((oldVal) -> { String newCommitTime; do { - if (commitTimeZone.equals(HoodieTimelineTimeZone.UTC.toString())) { + if (commitTimeZone.equals(HoodieTimelineTimeZone.UTC)) { LocalDateTime now = LocalDateTime.now(ZoneOffset.UTC); newCommitTime = now.format(MILLIS_INSTANT_TIME_FORMATTER); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java index 1a3d053e23acd..9741ceef3ede3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java @@ -77,8 +77,8 @@ public static List combine(List one, List another) { * NOTE: That values associated with overlapping keys from the second map, will override * values from the first one */ - public static Map combine(Map one, Map another) { - Map combined = new HashMap<>(one.size() + another.size()); + public static HashMap combine(Map one, Map another) { + HashMap combined = new HashMap<>(one.size() + another.size()); combined.putAll(one); combined.putAll(another); return combined; 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 dee91b2828aee..9970687abb4f2 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.exception.HoodieException; +import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -40,6 +41,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(); /** * Gets the commit action type for given write operation and table type. @@ -84,7 +86,8 @@ public static HoodieCommitMetadata buildMetadata(List writeStat if (extraMetadata.isPresent()) { extraMetadata.get().forEach(commitMetadata::addMetadata); } - commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaToStoreInCommit == null ? "" : schemaToStoreInCommit); + commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, (schemaToStoreInCommit == null || schemaToStoreInCommit.equals(NULL_SCHEMA_STR)) + ? "" : schemaToStoreInCommit); commitMetadata.setOperationType(operationType); return commitMetadata; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java new file mode 100644 index 0000000000000..738be514b2cbf --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CustomizedThreadFactory.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.jetbrains.annotations.NotNull; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicLong; + +/** + * A thread factory for creation of threads + */ +public class CustomizedThreadFactory implements ThreadFactory { + + private static final AtomicLong POOL_NUM = new AtomicLong(1); + private final AtomicLong threadNum = new AtomicLong(1); + + private final String threadName; + private final boolean daemon; + + public CustomizedThreadFactory() { + this("pool-" + POOL_NUM.getAndIncrement(), false); + } + + public CustomizedThreadFactory(String threadNamePrefix) { + this(threadNamePrefix, false); + } + + public CustomizedThreadFactory(String threadNamePrefix, boolean daemon) { + this.threadName = threadNamePrefix + "-thread-"; + this.daemon = daemon; + } + + @Override + public Thread newThread(@NotNull Runnable r) { + Thread runThread = new Thread(r); + runThread.setDaemon(daemon); + runThread.setName(threadName + threadNum.getAndIncrement()); + return runThread; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java index 1e320a552f024..29c42e3ea11ce 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/NetworkUtils.java @@ -23,6 +23,10 @@ import java.io.IOException; import java.net.DatagramSocket; import java.net.InetAddress; +import java.net.InterfaceAddress; +import java.net.NetworkInterface; +import java.net.SocketException; +import java.util.Enumeration; /** * A utility class for network. @@ -30,6 +34,21 @@ public class NetworkUtils { public static synchronized String getHostname() { + try { + Enumeration networkInterfaceEnumeration = NetworkInterface.getNetworkInterfaces(); + while (networkInterfaceEnumeration.hasMoreElements()) { + for (InterfaceAddress interfaceAddress : networkInterfaceEnumeration.nextElement().getInterfaceAddresses()) { + InetAddress address = interfaceAddress.getAddress(); + if (!address.isLinkLocalAddress() && !address.isLoopbackAddress() && !address.isAnyLocalAddress()) { + return address.getHostAddress(); + } + } + } + } catch (SocketException e) { + throw new HoodieException("Unable to find server port", e); + } + + // fallback try (DatagramSocket s = new DatagramSocket()) { // see https://stackoverflow.com/questions/9481865/getting-the-ip-address-of-the-current-machine-using-java // for details. 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 68b840a4794d6..d1e5e66083196 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 @@ -18,6 +18,7 @@ package org.apache.hudi.common.util.queue; +import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.DefaultSizeEstimator; import org.apache.hudi.common.util.Functions; import org.apache.hudi.common.util.Option; @@ -48,8 +49,10 @@ public class BoundedInMemoryExecutor { private static final Logger LOG = LogManager.getLogger(BoundedInMemoryExecutor.class); - // Executor service used for launching writer thread. - private final ExecutorService executorService; + // Executor service used for launching write thread. + private final ExecutorService producerExecutorService; + // Executor service used for launching read thread. + private final ExecutorService consumerExecutorService; // Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES. private final BoundedInMemoryQueue queue; // Producers @@ -60,28 +63,30 @@ public class BoundedInMemoryExecutor { private final Runnable preExecuteRunnable; public BoundedInMemoryExecutor(final long bufferLimitInBytes, final Iterator inputItr, - BoundedInMemoryQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { + BoundedInMemoryQueueConsumer consumer, Function transformFunction, Runnable preExecuteRunnable) { this(bufferLimitInBytes, new IteratorBasedQueueProducer<>(inputItr), Option.of(consumer), transformFunction, preExecuteRunnable); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer producer, - Option> consumer, final Function transformFunction) { + Option> consumer, final Function transformFunction) { this(bufferLimitInBytes, producer, consumer, transformFunction, Functions.noop()); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer producer, - Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { + Option> consumer, final Function transformFunction, Runnable preExecuteRunnable) { this(bufferLimitInBytes, Collections.singletonList(producer), consumer, transformFunction, new DefaultSizeEstimator<>(), preExecuteRunnable); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, - Option> consumer, final Function transformFunction, - final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { + Option> consumer, final Function transformFunction, + final SizeEstimator sizeEstimator, Runnable preExecuteRunnable) { this.producers = producers; this.consumer = consumer; this.preExecuteRunnable = preExecuteRunnable; - // Ensure single thread for each producer thread and one for consumer - this.executorService = Executors.newFixedThreadPool(producers.size() + 1); + // Ensure fixed thread for each producer thread + this.producerExecutorService = Executors.newFixedThreadPool(producers.size(), new CustomizedThreadFactory("producer")); + // Ensure single thread for consumer + this.consumerExecutorService = Executors.newSingleThreadExecutor(new CustomizedThreadFactory("consumer")); this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator); } @@ -92,7 +97,7 @@ public ExecutorCompletionService startProducers() { // Latch to control when and which producer thread will close the queue final CountDownLatch latch = new CountDownLatch(producers.size()); final ExecutorCompletionService completionService = - new ExecutorCompletionService(executorService); + new ExecutorCompletionService(producerExecutorService); producers.stream().map(producer -> { return completionService.submit(() -> { try { @@ -122,7 +127,7 @@ public ExecutorCompletionService startProducers() { */ private Future startConsumer() { return consumer.map(consumer -> { - return executorService.submit(() -> { + return consumerExecutorService.submit(() -> { LOG.info("starting consumer thread"); preExecuteRunnable.run(); try { @@ -143,7 +148,7 @@ private Future startConsumer() { */ public E execute() { try { - ExecutorCompletionService producerService = startProducers(); + startProducers(); Future future = startConsumer(); // Wait for consumer to be done return future.get(); @@ -161,7 +166,8 @@ public boolean isRemaining() { } public void shutdownNow() { - executorService.shutdownNow(); + producerExecutorService.shutdownNow(); + consumerExecutorService.shutdownNow(); } public BoundedInMemoryQueue getQueue() { 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 3c648f38defc6..2dce66e700479 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,10 +19,10 @@ package org.apache.hudi.metadata; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; 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; @@ -40,11 +40,15 @@ import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -143,9 +147,8 @@ public Map getAllFilesInPartitions(List partitions throws IOException { if (isMetadataTableEnabled) { try { - List partitionPaths = partitions.stream().map(entry -> new Path(entry)).collect(Collectors.toList()); - Map partitionsFilesMap = fetchAllFilesInPartitionPaths(partitionPaths); - return partitionsFilesMap; + List partitionPaths = partitions.stream().map(Path::new).collect(Collectors.toList()); + return fetchAllFilesInPartitionPaths(partitionPaths); } catch (Exception e) { throw new HoodieMetadataException("Failed to retrieve files in partition from metadata", e); } @@ -156,7 +159,7 @@ public Map getAllFilesInPartitions(List partitions } @Override - public Option getBloomFilter(final String partitionName, final String fileName) + public Option getBloomFilter(final String partitionName, final String fileName) throws HoodieMetadataException { if (!isBloomFilterIndexEnabled) { LOG.error("Metadata bloom filter index is disabled!"); @@ -164,7 +167,7 @@ public Option getBloomFilter(final String partitionName, final Strin } final Pair partitionFileName = Pair.of(partitionName, fileName); - Map, ByteBuffer> bloomFilters = getBloomFilters(Collections.singletonList(partitionFileName)); + Map, BloomFilter> bloomFilters = getBloomFilters(Collections.singletonList(partitionFileName)); if (bloomFilters.isEmpty()) { LOG.error("Meta index: missing bloom filter for partition: " + partitionName + ", file: " + fileName); return Option.empty(); @@ -175,7 +178,7 @@ public Option getBloomFilter(final String partitionName, final Strin } @Override - public Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + public Map, BloomFilter> getBloomFilters(final List> partitionNameFileNameList) throws HoodieMetadataException { if (!isBloomFilterIndexEnabled) { LOG.error("Metadata bloom filter index is disabled!"); @@ -202,7 +205,7 @@ public Map, ByteBuffer> getBloomFilters(final List m.updateMetrics(HoodieMetadataMetrics.LOOKUP_BLOOM_FILTERS_METADATA_STR, (timer.endTimer() / partitionIDFileIDStrings.size()))); - Map, ByteBuffer> partitionFileToBloomFilterMap = new HashMap<>(); + Map, BloomFilter> partitionFileToBloomFilterMap = new HashMap<>(); for (final Pair>> entry : hoodieRecordList) { if (entry.getRight().isPresent()) { final Option bloomFilterMetadata = @@ -210,7 +213,11 @@ public Map, ByteBuffer> getBloomFilters(final List, HoodieMetadataColumnStats> getColumnStats(final /** * Returns a list of all partitions. */ - protected List fetchAllPartitionPaths() throws IOException { + protected List fetchAllPartitionPaths() { HoodieTimer timer = new HoodieTimer().startTimer(); Option> hoodieRecord = getRecordByKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.getPartitionPath()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java index a4e5ea3539f17..1bb18bad16e40 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/FileSystemBackedTableMetadata.java @@ -19,6 +19,7 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; @@ -33,7 +34,6 @@ import org.apache.hudi.exception.HoodieMetadataException; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -143,13 +143,13 @@ public void reset() { // no-op } - public Option getBloomFilter(final String partitionName, final String fileName) + public Option getBloomFilter(final String partitionName, final String fileName) throws HoodieMetadataException { throw new HoodieMetadataException("Unsupported operation: getBloomFilter for " + fileName); } @Override - public Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + public Map, BloomFilter> getBloomFilters(final List> partitionNameFileNameList) throws HoodieMetadataException { throw new HoodieMetadataException("Unsupported operation: getBloomFilters!"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 221b52e77e674..c0ad8b14794bd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -18,18 +18,10 @@ package org.apache.hudi.metadata; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieMetadataBloomFilter; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieMetadataFileInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; -import org.apache.hudi.common.bloom.BloomFilterTypeCode; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieColumnRangeMetadata; @@ -37,13 +29,20 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.FileIndexID; import org.apache.hudi.common.util.hash.PartitionIndexID; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.io.storage.HoodieHFileReader; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -52,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -113,7 +113,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload recordOpt) { // This can be simplified using SpecificData.deepcopy once this bug is fixed // https://issues.apache.org/jira/browse/AVRO-1811 // - // NOTE: {@code HoodieMetadataRecord} has to always carry both "key" nad "type" fields + // NOTE: {@code HoodieMetadataRecord} has to always carry both "key" and "type" fields // for it to be handled appropriately, therefore these fields have to be reflected // in any (read-)projected schema key = record.get(KEY_FIELD_NAME).toString(); @@ -176,7 +176,7 @@ public HoodieMetadataPayload(Option recordOpt) { String.format("Valid %s record expected for type: %s", SCHEMA_FIELD_ID_COLUMN_STATS, METADATA_TYPE_COLUMN_STATS)); } else { columnStatMetadata = HoodieMetadataColumnStats.newBuilder() - .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_RESOURCE_NAME)) + .setFileName((String) columnStatsRecord.get(COLUMN_STATS_FIELD_FILE_NAME)) .setMinValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MIN_VALUE)) .setMaxValue((String) columnStatsRecord.get(COLUMN_STATS_FIELD_MAX_VALUE)) .setValueCount((Long) columnStatsRecord.get(COLUMN_STATS_FIELD_VALUE_COUNT)) @@ -239,10 +239,22 @@ public static HoodieRecord createPartitionFilesRecord(Str Option> filesAdded, Option> filesDeleted) { Map fileInfo = new HashMap<>(); - filesAdded.ifPresent( - m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false)))); - filesDeleted.ifPresent( - m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true)))); + filesAdded.ifPresent(filesMap -> + fileInfo.putAll( + filesMap.entrySet().stream().collect( + Collectors.toMap(Map.Entry::getKey, (entry) -> { + long fileSize = entry.getValue(); + // Assert that the file-size of the file being added is positive, since Hudi + // should not be creating empty files + checkState(fileSize > 0); + return new HoodieMetadataFileInfo(fileSize, false); + }))) + ); + filesDeleted.ifPresent(filesList -> + fileInfo.putAll( + filesList.stream().collect( + Collectors.toMap(Function.identity(), (ignored) -> new HoodieMetadataFileInfo(0L, true)))) + ); HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.getPartitionPath()); HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), METADATA_TYPE_FILE_LIST, fileInfo); @@ -262,33 +274,31 @@ public static HoodieRecord createPartitionFilesRecord(Str public static HoodieRecord createBloomFilterMetadataRecord(final String partitionName, final String baseFileName, final String timestamp, + final String bloomFilterType, final ByteBuffer bloomFilter, final boolean isDeleted) { - ValidationUtils.checkArgument(!baseFileName.contains(Path.SEPARATOR) + checkArgument(!baseFileName.contains(Path.SEPARATOR) && FSUtils.isBaseFile(new Path(baseFileName)), "Invalid base file '" + baseFileName + "' for MetaIndexBloomFilter!"); final String bloomFilterIndexKey = new PartitionIndexID(partitionName).asBase64EncodedString() .concat(new FileIndexID(baseFileName).asBase64EncodedString()); HoodieKey key = new HoodieKey(bloomFilterIndexKey, MetadataPartitionType.BLOOM_FILTERS.getPartitionPath()); - // TODO: HUDI-3203 Get the bloom filter type from the file HoodieMetadataBloomFilter metadataBloomFilter = - new HoodieMetadataBloomFilter(BloomFilterTypeCode.DYNAMIC_V0.name(), - timestamp, bloomFilter, isDeleted); - HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(), - metadataBloomFilter); + new HoodieMetadataBloomFilter(bloomFilterType, timestamp, bloomFilter, isDeleted); + HoodieMetadataPayload metadataPayload = new HoodieMetadataPayload(key.getRecordKey(), metadataBloomFilter); return new HoodieAvroRecord<>(key, metadataPayload); } @Override public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { - ValidationUtils.checkArgument(previousRecord.type == type, + checkArgument(previousRecord.type == type, "Cannot combine " + previousRecord.type + " with " + type); switch (type) { case METADATA_TYPE_PARTITION_LIST: case METADATA_TYPE_FILE_LIST: - Map combinedFileInfo = combineFilesystemMetadata(previousRecord); + Map combinedFileInfo = combineFileSystemMetadata(previousRecord); return new HoodieMetadataPayload(key, type, combinedFileInfo); case METADATA_TYPE_BLOOM_FILTER: HoodieMetadataBloomFilter combineBloomFilterMetadata = combineBloomFilterMetadata(previousRecord); @@ -301,11 +311,16 @@ public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) { } private HoodieMetadataBloomFilter combineBloomFilterMetadata(HoodieMetadataPayload previousRecord) { + // Bloom filters are always additive. No need to merge with previous bloom filter return this.bloomFilterMetadata; } private HoodieMetadataColumnStats combineColumnStatsMetadata(HoodieMetadataPayload previousRecord) { - return this.columnStatMetadata; + checkArgument(previousRecord.getColumnStatMetadata().isPresent()); + checkArgument(getColumnStatMetadata().isPresent()); + checkArgument(previousRecord.getColumnStatMetadata().get() + .getFileName().equals(this.columnStatMetadata.getFileName())); + return HoodieTableMetadataUtil.mergeColumnStats(previousRecord.getColumnStatMetadata().get(), this.columnStatMetadata); } @Override @@ -340,7 +355,7 @@ public Option getInsertValue(Schema schema) throws IOException { * Returns the list of filenames added as part of this record. */ public List getFilenames() { - return filterFileInfoEntries(false).map(e -> e.getKey()).sorted().collect(Collectors.toList()); + return filterFileInfoEntries(false).map(Map.Entry::getKey).sorted().collect(Collectors.toList()); } /** @@ -392,28 +407,53 @@ private Stream> filterFileInfoEntries( return filesystemMetadata.entrySet().stream().filter(e -> e.getValue().getIsDeleted() == isDeleted); } - private Map combineFilesystemMetadata(HoodieMetadataPayload previousRecord) { + private Map combineFileSystemMetadata(HoodieMetadataPayload previousRecord) { Map combinedFileInfo = new HashMap<>(); + + // First, add all files listed in the previous record if (previousRecord.filesystemMetadata != null) { combinedFileInfo.putAll(previousRecord.filesystemMetadata); } + // Second, merge in the files listed in the new record if (filesystemMetadata != null) { - filesystemMetadata.forEach((filename, fileInfo) -> { - // If the filename wasnt present then we carry it forward - if (!combinedFileInfo.containsKey(filename)) { - combinedFileInfo.put(filename, fileInfo); - } else { - if (fileInfo.getIsDeleted()) { - // file deletion - combinedFileInfo.remove(filename); - } else { - // file appends. - combinedFileInfo.merge(filename, fileInfo, (oldFileInfo, newFileInfo) -> { - return new HoodieMetadataFileInfo(oldFileInfo.getSize() + newFileInfo.getSize(), false); - }); - } - } + validatePayload(type, filesystemMetadata); + + filesystemMetadata.forEach((key, fileInfo) -> { + combinedFileInfo.merge(key, fileInfo, + // Combine previous record w/ the new one, new records taking precedence over + // the old one + // + // NOTE: That if previous listing contains the file that is being deleted by the tombstone + // record (`IsDeleted` = true) in the new one, we simply delete the file from the resulting + // listing as well as drop the tombstone itself. + // However, if file is not present in the previous record we have to persist tombstone + // record in the listing to make sure we carry forward information that this file + // was deleted. This special case could occur since the merging flow is 2-stage: + // - First we merge records from all of the delta log-files + // - Then we merge records from base-files with the delta ones (coming as a result + // of the previous step) + (oldFileInfo, newFileInfo) -> + // NOTE: We can’t assume that MT update records will be ordered the same way as actual + // FS operations (since they are not atomic), therefore MT record merging should be a + // _commutative_ & _associative_ operation (ie one that would work even in case records + // will get re-ordered), which is + // - Possible for file-sizes (since file-sizes will ever grow, we can simply + // take max of the old and new records) + // - Not possible for is-deleted flags* + // + // *However, we’re assuming that the case of concurrent write and deletion of the same + // file is _impossible_ -- it would only be possible with concurrent upsert and + // rollback operation (affecting the same log-file), which is implausible, b/c either + // of the following have to be true: + // - We’re appending to failed log-file (then the other writer is trying to + // rollback it concurrently, before it’s own write) + // - Rollback (of completed instant) is running concurrently with append (meaning + // that restore is running concurrently with a write, which is also nut supported + // currently) + newFileInfo.getIsDeleted() + ? null + : new HoodieMetadataFileInfo(Math.max(newFileInfo.getSize(), oldFileInfo.getSize()), false)); }); } @@ -480,8 +520,6 @@ public static Stream createColumnStatsRecords( .build()); return new HoodieAvroRecord<>(key, payload); }); - - } @Override @@ -494,9 +532,9 @@ public String toString() { if (type == METADATA_TYPE_BLOOM_FILTER) { checkState(getBloomFilterMetadata().isPresent()); sb.append("BloomFilter: {"); - sb.append("bloom size: " + getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); - sb.append("timestamp: " + getBloomFilterMetadata().get().getTimestamp()).append(", "); - sb.append("deleted: " + getBloomFilterMetadata().get().getIsDeleted()); + sb.append("bloom size: ").append(getBloomFilterMetadata().get().getBloomFilter().array().length).append(", "); + sb.append("timestamp: ").append(getBloomFilterMetadata().get().getTimestamp()).append(", "); + sb.append("deleted: ").append(getBloomFilterMetadata().get().getIsDeleted()); sb.append("}"); } if (type == METADATA_TYPE_COLUMN_STATS) { @@ -509,6 +547,14 @@ public String toString() { return sb.toString(); } + private static void validatePayload(int type, Map filesystemMetadata) { + if (type == METADATA_TYPE_FILE_LIST) { + filesystemMetadata.forEach((fileName, fileInfo) -> { + checkState(fileInfo.getIsDeleted() || fileInfo.getSize() > 0, "Existing files should have size > 0"); + }); + } + } + private static T getNestedFieldValue(GenericRecord record, String fieldName) { // NOTE: This routine is more lightweight than {@code HoodieAvroUtils.getNestedFieldVal} if (record.getSchema().getField(fieldName) == null) { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index 52fdbd993627f..841c5fc47e9ee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieMetadataColumnStats; +import org.apache.hudi.common.bloom.BloomFilter; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.SerializableConfiguration; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -31,7 +32,6 @@ import java.io.IOException; import java.io.Serializable; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -122,20 +122,20 @@ static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetad * * @param partitionName - Partition name * @param fileName - File name for which bloom filter needs to be retrieved - * @return BloomFilter byte buffer if available, otherwise empty + * @return BloomFilter if available, otherwise empty * @throws HoodieMetadataException */ - Option getBloomFilter(final String partitionName, final String fileName) + Option getBloomFilter(final String partitionName, final String fileName) throws HoodieMetadataException; /** * Get bloom filters for files from the metadata table index. * * @param partitionNameFileNameList - List of partition and file name pair for which bloom filters need to be retrieved - * @return Map of partition file name pair to its bloom filter byte buffer + * @return Map of partition file name pair to its bloom filter * @throws HoodieMetadataException */ - Map, ByteBuffer> getBloomFilters(final List> partitionNameFileNameList) + Map, BloomFilter> getBloomFilters(final List> partitionNameFileNameList) throws HoodieMetadataException; /** diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index e569baefb6f06..1a3739df2f8ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -19,6 +19,7 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieMetadataColumnStats; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.common.bloom.BloomFilter; @@ -40,34 +41,51 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ParquetUtils; +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.exception.HoodieIOException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import javax.annotation.Nonnull; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldValAsString; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.COLUMN_RANGE_MERGE_FUNCTION; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MAX; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.MIN; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.NULL_COUNT; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_SIZE; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.TOTAL_UNCOMPRESSED_SIZE; +import static org.apache.hudi.common.model.HoodieColumnRangeMetadata.Stats.VALUE_COUNT; import static org.apache.hudi.metadata.HoodieTableMetadata.EMPTY_PARTITION_NAME; import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME; @@ -103,37 +121,27 @@ public static void deleteMetadataTable(String basePath, HoodieEngineContext cont /** * Convert commit action to metadata records for the enabled partition types. * - * @param commitMetadata - Commit action metadata - * @param dataMetaClient - Meta client for the data table - * @param isMetaIndexColumnStatsForAllColumns - Do all columns need meta indexing? - * @param instantTime - Action instant time + * @param commitMetadata - Commit action metadata + * @param instantTime - Action instant time + * @param recordsGenerationParams - Parameters for the record generation * @return Map of partition to metadata records for the commit action */ public static Map> convertMetadataToRecords( - HoodieEngineContext context, List enabledPartitionTypes, - HoodieCommitMetadata commitMetadata, HoodieTableMetaClient dataMetaClient, - boolean isMetaIndexColumnStatsForAllColumns, String instantTime) { + HoodieEngineContext context, HoodieCommitMetadata commitMetadata, String instantTime, + MetadataRecordsGenerationParams recordsGenerationParams) { final Map> partitionToRecordsMap = new HashMap<>(); final HoodieData filesPartitionRecordsRDD = context.parallelize( convertMetadataToFilesPartitionRecords(commitMetadata, instantTime), 1); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { - final List metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(commitMetadata, - dataMetaClient, instantTime); - if (!metadataBloomFilterRecords.isEmpty()) { - final HoodieData metadataBloomFilterRecordsRDD = context.parallelize(metadataBloomFilterRecords, 1); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(context, commitMetadata, instantTime, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecords); } - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { - final List metadataColumnStats = convertMetadataToColumnStatsRecords(commitMetadata, context, - dataMetaClient, isMetaIndexColumnStatsForAllColumns, instantTime); - if (!metadataColumnStats.isEmpty()) { - final HoodieData metadataColumnStatsRDD = context.parallelize(metadataColumnStats, 1); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData metadataColumnStatsRDD = convertMetadataToColumnStatsRecords(commitMetadata, context, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); } return partitionToRecordsMap; } @@ -147,134 +155,148 @@ public static Map> convertMetada */ public static List convertMetadataToFilesPartitionRecords(HoodieCommitMetadata commitMetadata, String instantTime) { - List records = new LinkedList<>(); - List allPartitions = new LinkedList<>(); - commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { - final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; - allPartitions.add(partition); - - Map newFiles = new HashMap<>(writeStats.size()); - writeStats.forEach(hoodieWriteStat -> { - String pathWithPartition = hoodieWriteStat.getPath(); - if (pathWithPartition == null) { - // Empty partition - LOG.warn("Unable to find path in write stat to update metadata table " + hoodieWriteStat); - return; - } - - int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : partition.length() + 1; - String filename = pathWithPartition.substring(offset); - long totalWriteBytes = newFiles.containsKey(filename) - ? newFiles.get(filename) + hoodieWriteStat.getTotalWriteBytes() - : hoodieWriteStat.getTotalWriteBytes(); - newFiles.put(filename, totalWriteBytes); - }); - // New files added to a partition - HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord( - partition, Option.of(newFiles), Option.empty()); - records.add(record); - }); + List records = new ArrayList<>(commitMetadata.getPartitionToWriteStats().size()); + + // Add record bearing partitions list + ArrayList partitionsList = new ArrayList<>(commitMetadata.getPartitionToWriteStats().keySet()); + + records.add(HoodieMetadataPayload.createPartitionListRecord(partitionsList)); + + // Update files listing records for each individual partition + List> updatedPartitionFilesRecords = + commitMetadata.getPartitionToWriteStats().entrySet() + .stream() + .map(entry -> { + String partitionStatName = entry.getKey(); + List writeStats = entry.getValue(); + + String partition = getPartition(partitionStatName); + + HashMap updatedFilesToSizesMapping = + writeStats.stream().reduce(new HashMap<>(writeStats.size()), + (map, stat) -> { + String pathWithPartition = stat.getPath(); + if (pathWithPartition == null) { + // Empty partition + LOG.warn("Unable to find path in write stat to update metadata table " + stat); + return map; + } + + int offset = partition.equals(NON_PARTITIONED_NAME) + ? (pathWithPartition.startsWith("/") ? 1 : 0) + : partition.length() + 1; + String filename = pathWithPartition.substring(offset); + + // Since write-stats are coming in no particular order, if the same + // file have previously been appended to w/in the txn, we simply pick max + // of the sizes as reported after every write, since file-sizes are + // monotonically increasing (ie file-size never goes down, unless deleted) + map.merge(filename, stat.getFileSizeInBytes(), Math::max); + + return map; + }, + CollectionUtils::combine); + + return HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.of(updatedFilesToSizesMapping), + Option.empty()); + }) + .collect(Collectors.toList()); - // New partitions created - HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(new ArrayList<>(allPartitions)); - records.add(record); + records.addAll(updatedPartitionFilesRecords); LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType() + ". #partitions_updated=" + records.size()); + return records; } /** * Convert commit action metadata to bloom filter records. * - * @param commitMetadata - Commit action metadata - * @param dataMetaClient - Meta client for the data table - * @param instantTime - Action instant time - * @return List of metadata table records + * @param context - Engine context to use + * @param commitMetadata - Commit action metadata + * @param instantTime - Action instant time + * @param recordsGenerationParams - Parameters for bloom filter record generation + * @return HoodieData of metadata table records */ - public static List convertMetadataToBloomFilterRecords(HoodieCommitMetadata commitMetadata, - HoodieTableMetaClient dataMetaClient, - String instantTime) { - List records = new LinkedList<>(); - commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> { - final String partition = partitionStatName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionStatName; - Map newFiles = new HashMap<>(writeStats.size()); - writeStats.forEach(hoodieWriteStat -> { - // No action for delta logs - if (hoodieWriteStat instanceof HoodieDeltaWriteStat) { - return; - } + public static HoodieData convertMetadataToBloomFilterRecords( + HoodieEngineContext context, HoodieCommitMetadata commitMetadata, + String instantTime, MetadataRecordsGenerationParams recordsGenerationParams) { + final List allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() + .flatMap(entry -> entry.stream()).collect(Collectors.toList()); + if (allWriteStats.isEmpty()) { + return context.emptyHoodieData(); + } - String pathWithPartition = hoodieWriteStat.getPath(); - if (pathWithPartition == null) { - // Empty partition - LOG.error("Failed to find path in write stat to update metadata table " + hoodieWriteStat); - return; - } - int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : - partition.length() + 1; + final int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getBloomIndexParallelism()), 1); + HoodieData allWriteStatsRDD = context.parallelize(allWriteStats, parallelism); + return allWriteStatsRDD.flatMap(hoodieWriteStat -> { + final String partition = hoodieWriteStat.getPartitionPath(); - final String fileName = pathWithPartition.substring(offset); - if (!FSUtils.isBaseFile(new Path(fileName))) { - return; - } - ValidationUtils.checkState(!newFiles.containsKey(fileName), "Duplicate files in HoodieCommitMetadata"); + // For bloom filter index, delta writes do not change the base file bloom filter entries + if (hoodieWriteStat instanceof HoodieDeltaWriteStat) { + return Collections.emptyListIterator(); + } - final Path writeFilePath = new Path(dataMetaClient.getBasePath(), pathWithPartition); + String pathWithPartition = hoodieWriteStat.getPath(); + if (pathWithPartition == null) { + // Empty partition + LOG.error("Failed to find path in write stat to update metadata table " + hoodieWriteStat); + return Collections.emptyListIterator(); + } + int offset = partition.equals(NON_PARTITIONED_NAME) ? (pathWithPartition.startsWith("/") ? 1 : 0) : + partition.length() + 1; + + final String fileName = pathWithPartition.substring(offset); + if (!FSUtils.isBaseFile(new Path(fileName))) { + return Collections.emptyListIterator(); + } + + final Path writeFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); + try (HoodieFileReader fileReader = + HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) { try { - HoodieFileReader fileReader = - HoodieFileReaderFactory.getFileReader(dataMetaClient.getHadoopConf(), writeFilePath); - try { - final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); - if (fileBloomFilter == null) { - LOG.error("Failed to read bloom filter for " + writeFilePath); - return; - } - ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); - HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( - partition, fileName, instantTime, bloomByteBuffer, false); - records.add(record); - } catch (Exception e) { + final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); + if (fileBloomFilter == null) { LOG.error("Failed to read bloom filter for " + writeFilePath); - return; + return Collections.emptyListIterator(); } + ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); + HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( + partition, fileName, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); + return Collections.singletonList(record).iterator(); + } catch (Exception e) { + LOG.error("Failed to read bloom filter for " + writeFilePath); + return Collections.emptyListIterator(); + } finally { fileReader.close(); - } catch (IOException e) { - LOG.error("Failed to get bloom filter for file: " + writeFilePath + ", write stat: " + hoodieWriteStat); } - }); + } catch (IOException e) { + LOG.error("Failed to get bloom filter for file: " + writeFilePath + ", write stat: " + hoodieWriteStat); + } + return Collections.emptyListIterator(); }); - - return records; } /** * Convert the clean action to metadata records. */ public static Map> convertMetadataToRecords( - HoodieEngineContext engineContext, List enabledPartitionTypes, - HoodieCleanMetadata cleanMetadata, HoodieTableMetaClient dataMetaClient, String instantTime) { + HoodieEngineContext engineContext, HoodieCleanMetadata cleanMetadata, + MetadataRecordsGenerationParams recordsGenerationParams, String instantTime) { final Map> partitionToRecordsMap = new HashMap<>(); final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( convertMetadataToFilesPartitionRecords(cleanMetadata, instantTime), 1); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { - final List metadataBloomFilterRecords = convertMetadataToBloomFilterRecords(cleanMetadata, - engineContext, instantTime); - if (!metadataBloomFilterRecords.isEmpty()) { - final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData metadataBloomFilterRecordsRDD = convertMetadataToBloomFilterRecords(cleanMetadata, engineContext, instantTime, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); } - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { - final List metadataColumnStats = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, - dataMetaClient); - if (!metadataColumnStats.isEmpty()) { - final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData metadataColumnStatsRDD = convertMetadataToColumnStatsRecords(cleanMetadata, engineContext, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); } return partitionToRecordsMap; @@ -292,7 +314,7 @@ public static List convertMetadataToFilesPartitionRecords(HoodieCl List records = new LinkedList<>(); int[] fileDeleteCount = {0}; cleanMetadata.getPartitionMetadata().forEach((partitionName, partitionMetadata) -> { - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + final String partition = getPartition(partitionName); // Files deleted from a partition List deletedFiles = partitionMetadata.getDeletePathPatterns(); HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(), @@ -310,14 +332,16 @@ public static List convertMetadataToFilesPartitionRecords(HoodieCl /** * Convert clean metadata to bloom filter index records. * - * @param cleanMetadata - Clean action metadata - * @param engineContext - Engine context - * @param instantTime - Clean action instant time + * @param cleanMetadata - Clean action metadata + * @param engineContext - Engine context + * @param instantTime - Clean action instant time + * @param recordsGenerationParams - Parameters for bloom filter record generation * @return List of bloom filter index records for the clean metadata */ - public static List convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata, - HoodieEngineContext engineContext, - String instantTime) { + public static HoodieData convertMetadataToBloomFilterRecords(HoodieCleanMetadata cleanMetadata, + HoodieEngineContext engineContext, + String instantTime, + MetadataRecordsGenerationParams recordsGenerationParams) { List> deleteFileList = new ArrayList<>(); cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { // Files deleted from a partition @@ -330,23 +354,24 @@ public static List convertMetadataToBloomFilterRecords(HoodieClean }); }); - return engineContext.map(deleteFileList, deleteFileInfo -> { - return HoodieMetadataPayload.createBloomFilterMetadataRecord( - deleteFileInfo.getLeft(), deleteFileInfo.getRight(), instantTime, ByteBuffer.allocate(0), true); - }, 1).stream().collect(Collectors.toList()); + final int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getBloomIndexParallelism()), 1); + HoodieData> deleteFileListRDD = engineContext.parallelize(deleteFileList, parallelism); + return deleteFileListRDD.map(deleteFileInfoPair -> HoodieMetadataPayload.createBloomFilterMetadataRecord( + deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), instantTime, StringUtils.EMPTY_STRING, + ByteBuffer.allocate(0), true)); } /** * Convert clean metadata to column stats index records. * - * @param cleanMetadata - Clean action metadata - * @param engineContext - Engine context - * @param datasetMetaClient - data table meta client + * @param cleanMetadata - Clean action metadata + * @param engineContext - Engine context + * @param recordsGenerationParams - Parameters for bloom filter record generation * @return List of column stats index records for the clean metadata */ - public static List convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata, - HoodieEngineContext engineContext, - HoodieTableMetaClient datasetMetaClient) { + public static HoodieData convertMetadataToColumnStatsRecords(HoodieCleanMetadata cleanMetadata, + HoodieEngineContext engineContext, + MetadataRecordsGenerationParams recordsGenerationParams) { List> deleteFileList = new ArrayList<>(); cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { // Files deleted from a partition @@ -354,54 +379,42 @@ public static List convertMetadataToColumnStatsRecords(HoodieClean deletedFiles.forEach(entry -> deleteFileList.add(Pair.of(partition, entry))); }); - List latestColumns = getLatestColumns(datasetMetaClient); - return engineContext.flatMap(deleteFileList, - deleteFileInfo -> { - if (deleteFileInfo.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { - return getColumnStats(deleteFileInfo.getKey(), deleteFileInfo.getValue(), datasetMetaClient, - latestColumns, true); - } - return Stream.empty(); - }, 1).stream().collect(Collectors.toList()); + final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); + final int parallelism = Math.max(Math.min(deleteFileList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + HoodieData> deleteFileListRDD = engineContext.parallelize(deleteFileList, parallelism); + return deleteFileListRDD.flatMap(deleteFileInfoPair -> { + if (deleteFileInfoPair.getRight().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + return getColumnStats(deleteFileInfoPair.getLeft(), deleteFileInfoPair.getRight(), recordsGenerationParams.getDataMetaClient(), columnsToIndex, true).iterator(); + } + return Collections.emptyListIterator(); + }); } /** * Convert restore action metadata to metadata table records. */ public static Map> convertMetadataToRecords( - HoodieEngineContext engineContext, List enabledPartitionTypes, - HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata, - HoodieTableMetaClient dataMetaClient, String instantTime, Option lastSyncTs) { + HoodieEngineContext engineContext, HoodieActiveTimeline metadataTableTimeline, HoodieRestoreMetadata restoreMetadata, + MetadataRecordsGenerationParams recordsGenerationParams, String instantTime, Option lastSyncTs) { final Map> partitionToRecordsMap = new HashMap<>(); final Map> partitionToAppendedFiles = new HashMap<>(); final Map> partitionToDeletedFiles = new HashMap<>(); - processRestoreMetadata(metadataTableTimeline, restoreMetadata, - partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs); - - final HoodieData filesPartitionRecordsRDD = engineContext.parallelize( - convertFilesToFilesPartitionRecords(partitionToDeletedFiles, - partitionToAppendedFiles, instantTime, "Restore"), 1); + processRestoreMetadata(metadataTableTimeline, restoreMetadata, partitionToAppendedFiles, partitionToDeletedFiles, lastSyncTs); + final HoodieData filesPartitionRecordsRDD = + engineContext.parallelize(convertFilesToFilesPartitionRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore"), 1); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecordsRDD); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { - final List metadataBloomFilterRecords = convertFilesToBloomFilterRecords( - engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); - if (!metadataBloomFilterRecords.isEmpty()) { - final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData metadataBloomFilterRecordsRDD = + convertFilesToBloomFilterRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams, instantTime); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); } - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { - final List metadataColumnStats = convertFilesToColumnStatsRecords( - engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); - if (!metadataColumnStats.isEmpty()) { - final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); } - return partitionToRecordsMap; } @@ -417,44 +430,35 @@ private static void processRestoreMetadata(HoodieActiveTimeline metadataTableTim Map> partitionToAppendedFiles, Map> partitionToDeletedFiles, Option lastSyncTs) { - restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> { - rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, - partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs)); - }); + restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, + partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs))); } /** * Convert rollback action metadata to metadata table records. */ public static Map> convertMetadataToRecords( - HoodieEngineContext engineContext, List enabledPartitionTypes, - HoodieActiveTimeline metadataTableTimeline, HoodieRollbackMetadata rollbackMetadata, - HoodieTableMetaClient dataMetaClient, String instantTime, Option lastSyncTs, boolean wasSynced) { + HoodieEngineContext engineContext, HoodieActiveTimeline metadataTableTimeline, + HoodieRollbackMetadata rollbackMetadata, MetadataRecordsGenerationParams recordsGenerationParams, + String instantTime, Option lastSyncTs, boolean wasSynced) { final Map> partitionToRecordsMap = new HashMap<>(); - Map> partitionToDeletedFiles = new HashMap<>(); Map> partitionToAppendedFiles = new HashMap<>(); - List filesPartitionRecords = convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata, - partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced); + + List filesPartitionRecords = + convertMetadataToRollbackRecords(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, instantTime, lastSyncTs, wasSynced); final HoodieData rollbackRecordsRDD = engineContext.parallelize(filesPartitionRecords, 1); partitionToRecordsMap.put(MetadataPartitionType.FILES, rollbackRecordsRDD); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { - final List metadataBloomFilterRecords = convertFilesToBloomFilterRecords( - engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); - if (!metadataBloomFilterRecords.isEmpty()) { - final HoodieData metadataBloomFilterRecordsRDD = engineContext.parallelize(metadataBloomFilterRecords, 1); - partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.BLOOM_FILTERS)) { + final HoodieData metadataBloomFilterRecordsRDD = + convertFilesToBloomFilterRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams, instantTime); + partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, metadataBloomFilterRecordsRDD); } - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { - final List metadataColumnStats = convertFilesToColumnStatsRecords( - engineContext, dataMetaClient, partitionToDeletedFiles, partitionToAppendedFiles, instantTime); - if (!metadataColumnStats.isEmpty()) { - final HoodieData metadataColumnStatsRDD = engineContext.parallelize(metadataColumnStats, 1); - partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); - } + if (recordsGenerationParams.getEnabledPartitionTypes().contains(MetadataPartitionType.COLUMN_STATS)) { + final HoodieData metadataColumnStatsRDD = convertFilesToColumnStatsRecords(engineContext, partitionToDeletedFiles, partitionToAppendedFiles, recordsGenerationParams); + partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, metadataColumnStatsRDD); } return partitionToRecordsMap; @@ -575,7 +579,7 @@ private static List convertFilesToFilesPartitionRecords(Map { fileChangeCount[0] += deletedFiles.size(); - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + final String partition = getPartition(partitionName); Option> filesAdded = Option.empty(); if (partitionToAppendedFiles.containsKey(partitionName)) { @@ -588,7 +592,7 @@ private static List convertFilesToFilesPartitionRecords(Map { - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; + final String partition = getPartition(partitionName); fileChangeCount[1] += appendedFileMap.size(); // Validate that no appended file has been deleted @@ -609,82 +613,133 @@ private static List convertFilesToFilesPartitionRecords(Map convertFilesToBloomFilterRecords(HoodieEngineContext engineContext, - HoodieTableMetaClient dataMetaClient, - Map> partitionToDeletedFiles, - Map> partitionToAppendedFiles, - String instantTime) { - List records = new LinkedList<>(); - partitionToDeletedFiles.forEach((partitionName, deletedFileList) -> deletedFileList.forEach(deletedFile -> { - if (!FSUtils.isBaseFile(new Path(deletedFile))) { - return; - } + static String getPartition(@Nonnull String path) { + return EMPTY_PARTITION_NAME.equals(path) ? NON_PARTITIONED_NAME : path; + } - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; - records.add(HoodieMetadataPayload.createBloomFilterMetadataRecord( - partition, deletedFile, instantTime, ByteBuffer.allocate(0), true)); - })); + /** + * Convert added and deleted files metadata to bloom filter index records. + */ + public static HoodieData convertFilesToBloomFilterRecords(HoodieEngineContext engineContext, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + MetadataRecordsGenerationParams recordsGenerationParams, + String instantTime) { + HoodieData allRecordsRDD = engineContext.emptyHoodieData(); + + List>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet() + .stream().map(e -> Pair.of(e.getKey(), e.getValue())).collect(Collectors.toList()); + int parallelism = Math.max(Math.min(partitionToDeletedFilesList.size(), recordsGenerationParams.getBloomIndexParallelism()), 1); + HoodieData>> partitionToDeletedFilesRDD = engineContext.parallelize(partitionToDeletedFilesList, parallelism); + + HoodieData deletedFilesRecordsRDD = partitionToDeletedFilesRDD.flatMap(partitionToDeletedFilesPair -> { + final String partitionName = partitionToDeletedFilesPair.getLeft(); + final List deletedFileList = partitionToDeletedFilesPair.getRight(); + return deletedFileList.stream().flatMap(deletedFile -> { + if (!FSUtils.isBaseFile(new Path(deletedFile))) { + return Stream.empty(); + } - partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> { - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; - appendedFileMap.forEach((appendedFile, length) -> { + final String partition = getPartition(partitionName); + return Stream.of(HoodieMetadataPayload.createBloomFilterMetadataRecord( + partition, deletedFile, instantTime, StringUtils.EMPTY_STRING, ByteBuffer.allocate(0), true)); + }).iterator(); + }); + allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD); + + List>> partitionToAppendedFilesList = partitionToAppendedFiles.entrySet() + .stream().map(entry -> Pair.of(entry.getKey(), entry.getValue())).collect(Collectors.toList()); + parallelism = Math.max(Math.min(partitionToAppendedFilesList.size(), recordsGenerationParams.getBloomIndexParallelism()), 1); + HoodieData>> partitionToAppendedFilesRDD = engineContext.parallelize(partitionToAppendedFilesList, parallelism); + + HoodieData appendedFilesRecordsRDD = partitionToAppendedFilesRDD.flatMap(partitionToAppendedFilesPair -> { + final String partitionName = partitionToAppendedFilesPair.getLeft(); + final Map appendedFileMap = partitionToAppendedFilesPair.getRight(); + final String partition = getPartition(partitionName); + return appendedFileMap.entrySet().stream().flatMap(appendedFileLengthPairEntry -> { + final String appendedFile = appendedFileLengthPairEntry.getKey(); if (!FSUtils.isBaseFile(new Path(appendedFile))) { - return; + return Stream.empty(); } final String pathWithPartition = partitionName + "/" + appendedFile; - final Path appendedFilePath = new Path(dataMetaClient.getBasePath(), pathWithPartition); - try { - HoodieFileReader fileReader = - HoodieFileReaderFactory.getFileReader(dataMetaClient.getHadoopConf(), appendedFilePath); + final Path appendedFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition); + try (HoodieFileReader fileReader = + HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) { final BloomFilter fileBloomFilter = fileReader.readBloomFilter(); if (fileBloomFilter == null) { LOG.error("Failed to read bloom filter for " + appendedFilePath); - return; + return Stream.empty(); } ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( - partition, appendedFile, instantTime, bloomByteBuffer, false); - records.add(record); - fileReader.close(); + partition, appendedFile, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); + return Stream.of(record); } catch (IOException e) { LOG.error("Failed to get bloom filter for file: " + appendedFilePath); } - }); + return Stream.empty(); + }).iterator(); }); - return records; + allRecordsRDD = allRecordsRDD.union(appendedFilesRecordsRDD); + + return allRecordsRDD; } /** - * Convert rollback action metadata to column stats index records. + * Convert added and deleted action metadata to column stats index records. */ - private static List convertFilesToColumnStatsRecords(HoodieEngineContext engineContext, - HoodieTableMetaClient datasetMetaClient, - Map> partitionToDeletedFiles, - Map> partitionToAppendedFiles, - String instantTime) { - List records = new LinkedList<>(); - List latestColumns = getLatestColumns(datasetMetaClient); - partitionToDeletedFiles.forEach((partitionName, deletedFileList) -> deletedFileList.forEach(deletedFile -> { - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; - if (deletedFile.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + public static HoodieData convertFilesToColumnStatsRecords(HoodieEngineContext engineContext, + Map> partitionToDeletedFiles, + Map> partitionToAppendedFiles, + MetadataRecordsGenerationParams recordsGenerationParams) { + HoodieData allRecordsRDD = engineContext.emptyHoodieData(); + final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); + + final List>> partitionToDeletedFilesList = partitionToDeletedFiles.entrySet() + .stream().map(e -> Pair.of(e.getKey(), e.getValue())).collect(Collectors.toList()); + int parallelism = Math.max(Math.min(partitionToDeletedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + final HoodieData>> partitionToDeletedFilesRDD = engineContext.parallelize(partitionToDeletedFilesList, parallelism); + + HoodieData deletedFilesRecordsRDD = partitionToDeletedFilesRDD.flatMap(partitionToDeletedFilesPair -> { + final String partitionName = partitionToDeletedFilesPair.getLeft(); + final String partition = getPartition(partitionName); + final List deletedFileList = partitionToDeletedFilesPair.getRight(); + + return deletedFileList.stream().flatMap(deletedFile -> { final String filePathWithPartition = partitionName + "/" + deletedFile; - records.addAll(getColumnStats(partition, filePathWithPartition, datasetMetaClient, - latestColumns, true).collect(Collectors.toList())); - } - })); - - partitionToAppendedFiles.forEach((partitionName, appendedFileMap) -> appendedFileMap.forEach( - (appendedFile, size) -> { - final String partition = partitionName.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionName; - if (appendedFile.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { - final String filePathWithPartition = partitionName + "/" + appendedFile; - records.addAll(getColumnStats(partition, filePathWithPartition, datasetMetaClient, - latestColumns, false).collect(Collectors.toList())); - } - })); - return records; + return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), columnsToIndex, true); + }).iterator(); + }); + allRecordsRDD = allRecordsRDD.union(deletedFilesRecordsRDD); + + final List>> partitionToAppendedFilesList = partitionToAppendedFiles.entrySet() + .stream().map(entry -> Pair.of(entry.getKey(), entry.getValue())).collect(Collectors.toList()); + parallelism = Math.max(Math.min(partitionToAppendedFilesList.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + final HoodieData>> partitionToAppendedFilesRDD = engineContext.parallelize(partitionToAppendedFilesList, parallelism); + + HoodieData appendedFilesRecordsRDD = partitionToAppendedFilesRDD.flatMap(partitionToAppendedFilesPair -> { + final String partitionName = partitionToAppendedFilesPair.getLeft(); + final String partition = getPartition(partitionName); + final Map appendedFileMap = partitionToAppendedFilesPair.getRight(); + + return appendedFileMap.entrySet().stream().flatMap(appendedFileNameLengthEntry -> { + if (!FSUtils.isBaseFile(new Path(appendedFileNameLengthEntry.getKey())) + || !appendedFileNameLengthEntry.getKey().endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { + return Stream.empty(); + } + final String filePathWithPartition = partitionName + "/" + appendedFileNameLengthEntry.getKey(); + return getColumnStats(partition, filePathWithPartition, recordsGenerationParams.getDataMetaClient(), columnsToIndex, false); + }).iterator(); + + }); + allRecordsRDD = allRecordsRDD.union(appendedFilesRecordsRDD); + + return allRecordsRDD; } /** @@ -749,7 +804,7 @@ public static HoodieTableFileSystemView getFileSystemView(HoodieTableMetaClient if (timeline.empty()) { final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime()); - timeline = new HoodieDefaultTimeline(Arrays.asList(instant).stream(), metaClient.getActiveTimeline()::getInstantDetails); + timeline = new HoodieDefaultTimeline(Stream.of(instant), metaClient.getActiveTimeline()::getInstantDetails); } return new HoodieTableFileSystemView(metaClient, timeline); } @@ -777,20 +832,16 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta } else { fileSliceStream = fsView.getLatestFileSlices(partition); } - return fileSliceStream.sorted((s1, s2) -> s1.getFileId().compareTo(s2.getFileId())).collect(Collectors.toList()); + return fileSliceStream.sorted(Comparator.comparing(FileSlice::getFileId)).collect(Collectors.toList()); } - public static List convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, - HoodieEngineContext engineContext, - HoodieTableMetaClient dataMetaClient, - boolean isMetaIndexColumnStatsForAllColumns, - String instantTime) { - + public static HoodieData convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, + HoodieEngineContext engineContext, + MetadataRecordsGenerationParams recordsGenerationParams) { try { List allWriteStats = commitMetadata.getPartitionToWriteStats().values().stream() .flatMap(entry -> entry.stream()).collect(Collectors.toList()); - return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, dataMetaClient, allWriteStats, - isMetaIndexColumnStatsForAllColumns); + return HoodieTableMetadataUtil.createColumnStatsFromWriteStats(engineContext, allWriteStats, recordsGenerationParams); } catch (Exception e) { throw new HoodieException("Failed to generate column stats records for metadata table ", e); } @@ -799,30 +850,20 @@ public static List convertMetadataToColumnStatsRecords(HoodieCommi /** * Create column stats from write status. * - * @param engineContext - Engine context - * @param datasetMetaClient - Dataset meta client - * @param allWriteStats - Write status to convert - * @param isMetaIndexColumnStatsForAllColumns - Are all columns enabled for indexing + * @param engineContext - Engine context + * @param allWriteStats - Write status to convert + * @param recordsGenerationParams - Parameters for columns stats record generation */ - public static List createColumnStatsFromWriteStats(HoodieEngineContext engineContext, - HoodieTableMetaClient datasetMetaClient, - List allWriteStats, - boolean isMetaIndexColumnStatsForAllColumns) throws Exception { + public static HoodieData createColumnStatsFromWriteStats(HoodieEngineContext engineContext, + List allWriteStats, + MetadataRecordsGenerationParams recordsGenerationParams) { if (allWriteStats.isEmpty()) { - return Collections.emptyList(); + return engineContext.emptyHoodieData(); } - - List prunedWriteStats = allWriteStats.stream().filter(writeStat -> { - return !(writeStat instanceof HoodieDeltaWriteStat); - }).collect(Collectors.toList()); - if (prunedWriteStats.isEmpty()) { - return Collections.emptyList(); - } - - return engineContext.flatMap(prunedWriteStats, - writeStat -> translateWriteStatToColumnStats(writeStat, datasetMetaClient, - getLatestColumns(datasetMetaClient, isMetaIndexColumnStatsForAllColumns)), - prunedWriteStats.size()); + final List columnsToIndex = getColumnsToIndex(recordsGenerationParams.getDataMetaClient(), recordsGenerationParams.isAllColumnStatsIndexEnabled()); + final int parallelism = Math.max(Math.min(allWriteStats.size(), recordsGenerationParams.getColumnStatsIndexParallelism()), 1); + HoodieData allWriteStatsRDD = engineContext.parallelize(allWriteStats, parallelism); + return allWriteStatsRDD.flatMap(writeStat -> translateWriteStatToColumnStats(writeStat, recordsGenerationParams.getDataMetaClient(), columnsToIndex).iterator()); } /** @@ -831,10 +872,10 @@ public static List createColumnStatsFromWriteStats(HoodieEngineCon * @param datasetMetaClient - Data table meta client * @param isMetaIndexColumnStatsForAllColumns - Is column stats indexing enabled for all columns */ - private static List getLatestColumns(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) { + private static List getColumnsToIndex(HoodieTableMetaClient datasetMetaClient, boolean isMetaIndexColumnStatsForAllColumns) { if (!isMetaIndexColumnStatsForAllColumns || datasetMetaClient.getCommitsTimeline().filterCompletedInstants().countInstants() < 1) { - return Collections.singletonList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp()); + return Arrays.asList(datasetMetaClient.getTableConfig().getRecordKeyFieldProp().split(",")); } TableSchemaResolver schemaResolver = new TableSchemaResolver(datasetMetaClient); @@ -848,27 +889,42 @@ private static List getLatestColumns(HoodieTableMetaClient datasetMetaCl } } - private static List getLatestColumns(HoodieTableMetaClient datasetMetaClient) { - return getLatestColumns(datasetMetaClient, false); + public static HoodieMetadataColumnStats mergeColumnStats(HoodieMetadataColumnStats oldColumnStats, HoodieMetadataColumnStats newColumnStats) { + ValidationUtils.checkArgument(oldColumnStats.getFileName().equals(newColumnStats.getFileName())); + if (newColumnStats.getIsDeleted()) { + return newColumnStats; + } + return HoodieMetadataColumnStats.newBuilder() + .setFileName(newColumnStats.getFileName()) + .setMinValue(Stream.of(oldColumnStats.getMinValue(), newColumnStats.getMinValue()).filter(Objects::nonNull).min(Comparator.naturalOrder()).orElse(null)) + .setMaxValue(Stream.of(oldColumnStats.getMinValue(), newColumnStats.getMinValue()).filter(Objects::nonNull).max(Comparator.naturalOrder()).orElse(null)) + .setValueCount(oldColumnStats.getValueCount() + newColumnStats.getValueCount()) + .setNullCount(oldColumnStats.getNullCount() + newColumnStats.getNullCount()) + .setTotalSize(oldColumnStats.getTotalSize() + newColumnStats.getTotalSize()) + .setTotalUncompressedSize(oldColumnStats.getTotalUncompressedSize() + newColumnStats.getTotalUncompressedSize()) + .setIsDeleted(newColumnStats.getIsDeleted()) + .build(); } public static Stream translateWriteStatToColumnStats(HoodieWriteStat writeStat, HoodieTableMetaClient datasetMetaClient, - List latestColumns) { - return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, latestColumns, false); - + List columnsToIndex) { + if (writeStat instanceof HoodieDeltaWriteStat && ((HoodieDeltaWriteStat) writeStat).getRecordsStats().isPresent()) { + Map> columnRangeMap = ((HoodieDeltaWriteStat) writeStat).getRecordsStats().get().getStats(); + List> columnRangeMetadataList = new ArrayList<>(columnRangeMap.values()); + return HoodieMetadataPayload.createColumnStatsRecords(writeStat.getPartitionPath(), columnRangeMetadataList, false); + } + return getColumnStats(writeStat.getPartitionPath(), writeStat.getPath(), datasetMetaClient, columnsToIndex,false); } private static Stream getColumnStats(final String partitionPath, final String filePathWithPartition, HoodieTableMetaClient datasetMetaClient, - List columns, boolean isDeleted) { - final String partition = partitionPath.equals(EMPTY_PARTITION_NAME) ? NON_PARTITIONED_NAME : partitionPath; + List columnsToIndex, + boolean isDeleted) { + final String partition = getPartition(partitionPath); final int offset = partition.equals(NON_PARTITIONED_NAME) ? (filePathWithPartition.startsWith("/") ? 1 : 0) : partition.length() + 1; final String fileName = filePathWithPartition.substring(offset); - if (!FSUtils.isBaseFile(new Path(fileName))) { - return Stream.empty(); - } if (filePathWithPartition.endsWith(HoodieFileFormat.PARQUET.getFileExtension())) { List> columnRangeMetadataList = new ArrayList<>(); @@ -876,13 +932,13 @@ private static Stream getColumnStats(final String partitionPath, f if (!isDeleted) { try { columnRangeMetadataList = new ParquetUtils().readRangeFromParquetMetadata( - datasetMetaClient.getHadoopConf(), fullFilePath, columns); + datasetMetaClient.getHadoopConf(), fullFilePath, columnsToIndex); } catch (Exception e) { LOG.error("Failed to read column stats for " + fullFilePath, e); } } else { columnRangeMetadataList = - columns.stream().map(entry -> new HoodieColumnRangeMetadata(fileName, + columnsToIndex.stream().map(entry -> new HoodieColumnRangeMetadata(fileName, entry, null, null, 0, 0, 0, 0)) .collect(Collectors.toList()); } @@ -922,4 +978,72 @@ public static int getPartitionFileGroupCount(final MetadataPartitionType partiti } } + /** + * Accumulates column range metadata for the given field and updates the column range map. + * + * @param field - column for which statistics will be computed + * @param filePath - data file path + * @param columnRangeMap - old column range statistics, which will be merged in this computation + * @param columnToStats - map of column to map of each stat and its value + */ + public static void accumulateColumnRanges(Schema.Field field, String filePath, + Map> columnRangeMap, + Map> columnToStats) { + Map columnStats = columnToStats.get(field.name()); + HoodieColumnRangeMetadata columnRangeMetadata = new HoodieColumnRangeMetadata<>( + filePath, + field.name(), + String.valueOf(columnStats.get(MIN)), + String.valueOf(columnStats.get(MAX)), + Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()), + Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()), + Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()), + Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString()) + ); + columnRangeMap.merge(field.name(), columnRangeMetadata, COLUMN_RANGE_MERGE_FUNCTION); + } + + /** + * Aggregates column stats for each field. + * + * @param record - current record + * @param schema - write schema + * @param columnToStats - map of column to map of each stat and its value which gets updates in this method + * @param consistentLogicalTimestampEnabled - flag to deal with logical timestamp type when getting column value + */ + public static void aggregateColumnStats(IndexedRecord record, Schema schema, + Map> columnToStats, + boolean consistentLogicalTimestampEnabled) { + if (!(record instanceof GenericRecord)) { + throw new HoodieIOException("Record is not a generic type to get column range metadata!"); + } + + schema.getFields().forEach(field -> { + Map columnStats = columnToStats.getOrDefault(field.name(), new HashMap<>()); + final String fieldVal = getNestedFieldValAsString((GenericRecord) record, field.name(), true, consistentLogicalTimestampEnabled); + // update stats + final int fieldSize = fieldVal == null ? 0 : fieldVal.length(); + columnStats.put(TOTAL_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_SIZE, 0).toString()) + fieldSize); + columnStats.put(TOTAL_UNCOMPRESSED_SIZE, Long.parseLong(columnStats.getOrDefault(TOTAL_UNCOMPRESSED_SIZE, 0).toString()) + fieldSize); + + if (!StringUtils.isNullOrEmpty(fieldVal)) { + // set the min value of the field + if (!columnStats.containsKey(MIN)) { + columnStats.put(MIN, fieldVal); + } + if (fieldVal.compareTo(String.valueOf(columnStats.get(MIN))) < 0) { + columnStats.put(MIN, fieldVal); + } + // set the max value of the field + if (fieldVal.compareTo(String.valueOf(columnStats.getOrDefault(MAX, ""))) > 0) { + columnStats.put(MAX, fieldVal); + } + // increment non-null value count + columnStats.put(VALUE_COUNT, Long.parseLong(columnStats.getOrDefault(VALUE_COUNT, 0).toString()) + 1); + } else { + // increment null value count + columnStats.put(NULL_COUNT, Long.parseLong(columnStats.getOrDefault(NULL_COUNT, 0).toString()) + 1); + } + }); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java new file mode 100644 index 0000000000000..21d5b173b2338 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import java.io.Serializable; +import java.util.List; + +/** + * Encapsulates all parameters required to generate metadata index for enabled index types. + */ +public class MetadataRecordsGenerationParams implements Serializable { + + private final HoodieTableMetaClient dataMetaClient; + private final List enabledPartitionTypes; + private final String bloomFilterType; + private final int bloomIndexParallelism; + private final boolean isAllColumnStatsIndexEnabled; + private final int columnStatsIndexParallelism; + + MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism, + boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism) { + this.dataMetaClient = dataMetaClient; + this.enabledPartitionTypes = enabledPartitionTypes; + this.bloomFilterType = bloomFilterType; + this.bloomIndexParallelism = bloomIndexParallelism; + this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled; + this.columnStatsIndexParallelism = columnStatsIndexParallelism; + } + + public HoodieTableMetaClient getDataMetaClient() { + return dataMetaClient; + } + + public List getEnabledPartitionTypes() { + return enabledPartitionTypes; + } + + public String getBloomFilterType() { + return bloomFilterType; + } + + public boolean isAllColumnStatsIndexEnabled() { + return isAllColumnStatsIndexEnabled; + } + + public int getBloomIndexParallelism() { + return bloomIndexParallelism; + } + + public int getColumnStatsIndexParallelism() { + return columnStatsIndexParallelism; + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java index 87d4e746d81cb..c0896e723ea07 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java @@ -143,6 +143,26 @@ public void testGetEventTimeInMetadata(long eventTime) throws IOException { Long.parseLong(payload2.getMetadata().get().get(DefaultHoodieRecordPayload.METADATA_EVENT_TIME_KEY))); } + @Test + public void testEmptyProperty() throws IOException { + GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", "1"); + record1.put("partition", "partition0"); + record1.put("ts", 0L); + record1.put("_hoodie_is_deleted", false); + + GenericRecord record2 = new GenericData.Record(schema); + record2.put("id", "1"); + record2.put("partition", "partition0"); + record2.put("ts", 1L); + record2.put("_hoodie_is_deleted", false); + + DefaultHoodieRecordPayload payload = new DefaultHoodieRecordPayload(Option.of(record1)); + Properties properties = new Properties(); + payload.getInsertValue(schema, properties); + payload.combineAndGetUpdateValue(record2, schema, properties); + } + @ParameterizedTest @ValueSource(longs = {1L, 1612542030000L}) public void testGetEventTimeInMetadataForInserts(long eventTime) throws IOException { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/properties/TestOrderedProperties.java b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestOrderedProperties.java new file mode 100644 index 0000000000000..c75df04a2a633 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestOrderedProperties.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.common.properties; + +import org.apache.hudi.common.config.OrderedProperties; + +import org.junit.jupiter.api.Test; + +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class TestOrderedProperties { + + @Test + public void testPutPropertiesOrder() { + Properties properties = new OrderedProperties(); + properties.put("key0", "true"); + properties.put("key1", "false"); + properties.put("key2", "true"); + properties.put("key3", "false"); + properties.put("key4", "true"); + properties.put("key5", "true"); + properties.put("key6", "false"); + properties.put("key7", "true"); + properties.put("key8", "false"); + properties.put("key9", "true"); + + OrderedProperties typedProperties = new OrderedProperties(properties); + assertTypeProperties(typedProperties, 0); + } + + @Test + void testPutAllPropertiesOrder() { + Properties firstProp = new OrderedProperties(); + firstProp.put("key0", "true"); + firstProp.put("key1", "false"); + firstProp.put("key2", "true"); + + OrderedProperties firstProperties = new OrderedProperties(firstProp); + assertTypeProperties(firstProperties, 0); + + OrderedProperties secondProperties = new OrderedProperties(); + secondProperties.put("key3", "true"); + secondProperties.put("key4", "false"); + secondProperties.put("key5", "true"); + assertTypeProperties(secondProperties, 3); + + OrderedProperties thirdProperties = new OrderedProperties(); + thirdProperties.putAll(firstProp); + thirdProperties.putAll(secondProperties); + + assertEquals(3, firstProp.stringPropertyNames().size()); + assertEquals(3, secondProperties.stringPropertyNames().size()); + assertEquals(6, thirdProperties.stringPropertyNames().size()); + } + + private void assertTypeProperties(OrderedProperties typedProperties, int start) { + String[] props = typedProperties.stringPropertyNames().toArray(new String[0]); + for (int i = start; i < props.length; i++) { + assertEquals(String.format("key%d", i), props[i]); + } + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java index a3ba13ec14a2a..f93627525d990 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/properties/TestTypedProperties.java @@ -22,10 +22,11 @@ import org.junit.jupiter.api.Test; -import java.io.IOException; 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; public class TestTypedProperties { @Test @@ -79,58 +80,8 @@ public void testGetBoolean() { properties.put("key1", "true"); TypedProperties typedProperties = new TypedProperties(properties); - assertEquals(true, typedProperties.getBoolean("key1")); - assertEquals(true, typedProperties.getBoolean("key1", false)); - assertEquals(false, typedProperties.getBoolean("key2", false)); - } - - @Test - public void testPropertiesOrder() throws IOException { - Properties properties = new TypedProperties(); - properties.put("key0", "true"); - properties.put("key1", "false"); - properties.put("key2", "true"); - properties.put("key3", "false"); - properties.put("key4", "true"); - properties.put("key5", "true"); - properties.put("key6", "false"); - properties.put("key7", "true"); - properties.put("key8", "false"); - properties.put("key9", "true"); - - TypedProperties typedProperties = new TypedProperties(properties); - assertTypeProperties(typedProperties, 0); - } - - @Test - void testPutAllProperties() { - Properties firstProp = new TypedProperties(); - firstProp.put("key0", "true"); - firstProp.put("key1", "false"); - firstProp.put("key2", "true"); - - TypedProperties firstProperties = new TypedProperties(firstProp); - assertTypeProperties(firstProperties, 0); - - TypedProperties secondProperties = new TypedProperties(); - secondProperties.put("key3", "true"); - secondProperties.put("key4", "false"); - secondProperties.put("key5", "true"); - assertTypeProperties(secondProperties, 3); - - TypedProperties thirdProperties = new TypedProperties(); - thirdProperties.putAll(firstProp); - thirdProperties.putAll(secondProperties); - - assertEquals(3, firstProp.stringPropertyNames().size()); - assertEquals(3, secondProperties.stringPropertyNames().size()); - assertEquals(6, thirdProperties.stringPropertyNames().size()); - } - - private void assertTypeProperties(TypedProperties typedProperties, int start) { - String[] props = typedProperties.stringPropertyNames().toArray(new String[0]); - for (int i = start; i < props.length; i++) { - assertEquals(String.format("key%d", i), props[i]); - } + assertTrue(typedProperties.getBoolean("key1")); + assertTrue(typedProperties.getBoolean("key1", false)); + assertFalse(typedProperties.getBoolean("key2", false)); } } 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 2b1057fea7c19..8f5e5ae964f83 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 @@ -285,7 +285,7 @@ public static void createPartitionMetaFile(String basePath, String partitionPath public static void createBaseFile(String basePath, String partitionPath, String instantTime, String fileId) throws Exception { - createBaseFile(basePath, partitionPath, instantTime, fileId, 0); + createBaseFile(basePath, partitionPath, instantTime, fileId, 1); } public static void createBaseFile(String basePath, String partitionPath, String instantTime, String fileId, long length) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index f78312217eec2..6f49c69960fc1 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -1057,6 +1057,7 @@ public static List generateHoodieWriteStatForPartition(Map generateHoodieWriteStatForPartitionLogFiles writeStat.setPartitionPath(partition); writeStat.setPath(partition + "/" + fileName); writeStat.setTotalWriteBytes(fileIdInfo.getValue()[1]); + writeStat.setFileSizeInBytes(fileIdInfo.getValue()[1]); writeStats.add(writeStat); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java index ab77caa1bcb83..70d5a1bb3e9d6 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/SchemaTestUtil.java @@ -194,6 +194,10 @@ public static Schema getEvolvedSchema() throws IOException { return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avsc")); } + public static Schema getEvolvedCompatibleSchema() throws IOException { + return new Schema.Parser().parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved-compatible.avsc")); + } + public static List generateEvolvedTestRecords(int from, int limit) throws IOException, URISyntaxException { return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCustomizedThreadFactory.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCustomizedThreadFactory.java new file mode 100644 index 0000000000000..797643a2c994f --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCustomizedThreadFactory.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.locks.LockSupport; + +public class TestCustomizedThreadFactory { + + @Test + public void testThreadPrefix() throws ExecutionException, InterruptedException { + int nThreads = 100; + String threadNamePrefix = "consumer"; + ExecutorService executorService = Executors.newFixedThreadPool(nThreads, new CustomizedThreadFactory(threadNamePrefix)); + for (int i = 0; i < nThreads; i++) { + Future resultFuture = executorService.submit(() -> { + LockSupport.parkNanos(10000000L); + String name = Thread.currentThread().getName(); + return name.startsWith(threadNamePrefix); + }); + Boolean result = resultFuture.get(); + Assertions.assertTrue(result); + } + } + + @Test + public void testDefaultThreadPrefix() throws ExecutionException, InterruptedException { + int nThreads = 100; + String defaultThreadNamePrefix = "pool-1"; + ExecutorService executorService = Executors.newFixedThreadPool(nThreads, new CustomizedThreadFactory()); + for (int i = 0; i < nThreads; i++) { + Future resultFuture = executorService.submit(() -> { + LockSupport.parkNanos(10000000L); + String name = Thread.currentThread().getName(); + return name.startsWith(defaultThreadNamePrefix); + }); + Boolean result = resultFuture.get(); + Assertions.assertTrue(result); + } + } + + @Test + public void testDaemonThread() throws ExecutionException, InterruptedException { + int nThreads = 100; + String threadNamePrefix = "consumer"; + ExecutorService executorService = Executors.newFixedThreadPool(nThreads, new CustomizedThreadFactory(threadNamePrefix, true)); + for (int i = 0; i < nThreads; i++) { + Future resultFuture = executorService.submit(() -> { + LockSupport.parkNanos(10000000L); + String name = Thread.currentThread().getName(); + boolean daemon = Thread.currentThread().isDaemon(); + return name.startsWith(threadNamePrefix) && daemon; + }); + Boolean result = resultFuture.get(); + Assertions.assertTrue(result); + } + } +} diff --git a/hudi-common/src/test/resources/simple-test-doced.avsc b/hudi-common/src/test/resources/simple-test-doced.avsc new file mode 100644 index 0000000000000..f6b53aff8ee8a --- /dev/null +++ b/hudi-common/src/test/resources/simple-test-doced.avsc @@ -0,0 +1,27 @@ +/* + * 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. + */ +{ +"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string","doc":"name_comment"}, + {"name": "favorite_number", "type": "int","doc":"favorite_number_comment"}, + {"name": "favorite_color", "type": "string"} + ] +} diff --git a/hudi-common/src/test/resources/simple-test-evolved-compatible.avsc b/hudi-common/src/test/resources/simple-test-evolved-compatible.avsc new file mode 100644 index 0000000000000..09463fa310937 --- /dev/null +++ b/hudi-common/src/test/resources/simple-test-evolved-compatible.avsc @@ -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. + */ +{ +"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favorite_number", "type": "int"}, + {"name": "favorite_color", "type": "string"}, + {"name": "field1", "type": ["null", "string"], "default": null}, + {"name": "field2", "type": ["null", "string"], "default": null} + ] +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java index 195e430d0b1b3..65f07d7c7a83e 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/CleanFunction.java @@ -19,6 +19,8 @@ package org.apache.hudi.sink; import org.apache.hudi.client.HoodieFlinkWriteClient; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.utils.NonThrownExecutor; import org.apache.hudi.util.StreamerUtil; @@ -60,10 +62,14 @@ public CleanFunction(Configuration conf) { public void open(Configuration parameters) throws Exception { super.open(parameters); if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) { - // do not use the remote filesystem view because the async cleaning service - // local timeline is very probably to fall behind with the remote one. - this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext(), false); - this.executor = NonThrownExecutor.builder(LOG).build(); + this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext()); + this.executor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build(); + + if (conf.getString(FlinkOptions.OPERATION).equals(WriteOperationType.INSERT_OVERWRITE_TABLE.value())) { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + LOG.info(String.format("exec sync clean with instant time %s...", instantTime)); + executor.execute(() -> writeClient.clean(instantTime), "wait for sync cleaning finish"); + } } } @@ -101,6 +107,10 @@ public void initializeState(FunctionInitializationContext context) throws Except @Override public void close() throws Exception { + if (executor != null) { + executor.close(); + } + if (this.writeClient != null) { this.writeClient.close(); } diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 4782070e33760..c4f2e771c5baf 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -33,6 +32,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.event.CommitAckEvent; import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.sink.meta.CkpMetadata; import org.apache.hudi.sink.utils.HiveSyncContext; import org.apache.hudi.sink.utils.NonThrownExecutor; import org.apache.hudi.util.CompactionUtil; @@ -138,6 +138,16 @@ public class StreamWriteOperatorCoordinator */ private transient TableState tableState; + /** + * The checkpoint metadata. + */ + private CkpMetadata ckpMetadata; + + /** + * Current checkpoint. + */ + private long checkpointId = -1; + /** * Constructs a StreamingSinkOperatorCoordinator. * @@ -176,6 +186,8 @@ public void start() throws Exception { if (tableState.syncMetadata) { initMetadataSync(); } + this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), metaClient.getBasePath()); + this.ckpMetadata.bootstrap(this.metaClient); } @Override @@ -193,10 +205,14 @@ public void close() throws Exception { writeClient.close(); } this.eventBuffer = null; + if (this.ckpMetadata != null) { + this.ckpMetadata.close(); + } } @Override public void checkpointCoordinator(long checkpointId, CompletableFuture result) { + this.checkpointId = checkpointId; executor.execute( () -> { try { @@ -239,6 +255,15 @@ public void notifyCheckpointComplete(long checkpointId) { ); } + @Override + public void notifyCheckpointAborted(long checkpointId) { + if (checkpointId == this.checkpointId) { + executor.execute(() -> { + this.ckpMetadata.abortInstant(this.instant); + }, "abort instant %s", this.instant); + } + } + @Override public void resetToCheckpoint(long checkpointID, byte[] checkpointData) { // no operation @@ -337,12 +362,11 @@ private void addEventToBuffer(WriteMetadataEvent event) { } private void startInstant() { - final String instant = HoodieActiveTimeline.createNewInstantTime(); // put the assignment in front of metadata generation, // because the instant request from write task is asynchronous. - this.instant = instant; - this.writeClient.startCommitWithTime(instant, tableState.commitAction); + this.instant = this.writeClient.startCommit(tableState.commitAction, this.metaClient); this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant); + this.ckpMetadata.startInstant(this.instant); LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant, this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE)); } @@ -491,6 +515,7 @@ private void doCommit(String instant, List writeResults) { tableState.commitAction, partitionToReplacedFileIds); if (success) { reset(); + this.ckpMetadata.commitInstant(instant); LOG.info("Commit instant [{}] success!", instant); } else { throw new HoodieException(String.format("Commit instant [%s] failed!", instant)); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index 0f944c56577e2..cdb378f88885d 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -37,6 +36,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction; +import org.apache.hudi.sink.meta.CkpMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.format.FormatUtils; import org.apache.hudi.util.FlinkTables; @@ -83,6 +83,8 @@ public class BootstrapOperator> protected HoodieTable hoodieTable; + private CkpMetadata ckpMetadata; + protected final Configuration conf; protected transient org.apache.hadoop.conf.Configuration hadoopConf; @@ -101,8 +103,7 @@ public BootstrapOperator(Configuration conf) { @Override public void snapshotState(StateSnapshotContext context) throws Exception { - HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(this.conf); - lastInstantTime = StreamerUtil.getLastPendingInstant(metaClient); + lastInstantTime = this.ckpMetadata.lastPendingInstant(); instantState.update(Collections.singletonList(lastInstantTime)); } @@ -124,6 +125,7 @@ public void initializeState(StateInitializationContext context) throws Exception this.hadoopConf = StreamerUtil.getHadoopConf(); this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true); this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext()); + this.ckpMetadata = CkpMetadata.getInstance(hoodieTable.getMetaClient().getFs(), this.writeConfig.getBasePath()); this.aggregateManager = getRuntimeContext().getGlobalAggregateManager(); preLoadIndexRecords(); diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java index 4089907243c87..9b34c3edcd800 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/bulk/BulkInsertWriteFunction.java @@ -21,11 +21,11 @@ import org.apache.hudi.client.HoodieFlinkWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.sink.StreamWriteOperatorCoordinator; import org.apache.hudi.sink.common.AbstractWriteFunction; import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.sink.meta.CkpMetadata; import org.apache.hudi.sink.utils.TimeWait; import org.apache.hudi.util.StreamerUtil; @@ -79,11 +79,6 @@ public class BulkInsertWriteFunction */ private int taskID; - /** - * Meta Client. - */ - private transient HoodieTableMetaClient metaClient; - /** * Write Client. */ @@ -99,6 +94,11 @@ public class BulkInsertWriteFunction */ private transient OperatorEventGateway eventGateway; + /** + * Checkpoint metadata. + */ + private CkpMetadata ckpMetadata; + /** * Constructs a StreamingSinkFunction. * @@ -112,9 +112,9 @@ public BulkInsertWriteFunction(Configuration config, RowType rowType) { @Override public void open(Configuration parameters) throws IOException { this.taskID = getRuntimeContext().getIndexOfThisSubtask(); - this.metaClient = StreamerUtil.createMetaClient(this.config); this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext()); - this.initInstant = StreamerUtil.getLastPendingInstant(this.metaClient, false); + this.ckpMetadata = CkpMetadata.getInstance(config.getString(FlinkOptions.PATH)); + this.initInstant = lastPendingInstant(); sendBootstrapEvent(); initWriterHelper(); } @@ -187,7 +187,7 @@ private void sendBootstrapEvent() { * Returns the last pending instant time. */ protected String lastPendingInstant() { - return StreamerUtil.getLastPendingInstant(this.metaClient); + return this.ckpMetadata.lastPendingInstant(); } private String instantToWrite() { diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java b/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java index 0e7300591286f..f9cf938e44aee 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java @@ -26,6 +26,7 @@ import org.apache.hudi.sink.StreamWriteOperatorCoordinator; import org.apache.hudi.sink.event.CommitAckEvent; import org.apache.hudi.sink.event.WriteMetadataEvent; +import org.apache.hudi.sink.meta.CkpMetadata; import org.apache.hudi.sink.utils.TimeWait; import org.apache.hudi.util.StreamerUtil; @@ -114,6 +115,11 @@ public abstract class AbstractStreamWriteFunction */ protected List writeStatuses; + /** + * The checkpoint metadata. + */ + private transient CkpMetadata ckpMetadata; + /** * Constructs a StreamWriteFunctionBase. * @@ -135,6 +141,7 @@ public void initializeState(FunctionInitializationContext context) throws Except TypeInformation.of(WriteMetadataEvent.class) )); + this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), this.metaClient.getBasePath()); this.currentInstant = lastPendingInstant(); if (context.isRestored()) { restoreWriteMetadata(); @@ -217,7 +224,7 @@ public void handleOperatorEvent(OperatorEvent event) { * Returns the last pending instant time. */ protected String lastPendingInstant() { - return StreamerUtil.getLastPendingInstant(this.metaClient); + return this.ckpMetadata.lastPendingInstant(); } /** @@ -238,7 +245,7 @@ protected String instantToWrite(boolean hasData) { // wait condition: // 1. there is no inflight instant // 2. the inflight instant does not change and the checkpoint has buffering data - if (instant == null || (instant.equals(this.currentInstant) && hasData)) { + if (instant == null || (instant.equals(this.currentInstant) && hasData && !this.ckpMetadata.isAborted(instant))) { // sleep for a while timeWait.waitFor(); // refresh the inflight instant diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMessage.java b/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMessage.java new file mode 100644 index 0000000000000..1a27ae05c5190 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMessage.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.meta; + +import org.apache.hudi.common.util.ValidationUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.jetbrains.annotations.NotNull; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +/** + * A checkpoint message. + */ +public class CkpMessage implements Serializable, Comparable { + private static final long serialVersionUID = 1L; + + public static final Comparator COMPARATOR = Comparator.comparing(CkpMessage::getInstant) + .thenComparing(CkpMessage::getState); + + private final String instant; // the instant time + private final State state; // the checkpoint state + + public CkpMessage(String instant, String state) { + this.instant = instant; + this.state = State.valueOf(state); + } + + public CkpMessage(FileStatus fileStatus) { + String fileName = fileStatus.getPath().getName(); + String[] nameAndExt = fileName.split("\\."); + ValidationUtils.checkState(nameAndExt.length == 2); + String name = nameAndExt[0]; + String ext = nameAndExt[1]; + + this.instant = name; + this.state = State.valueOf(ext); + } + + public String getInstant() { + return instant; + } + + public State getState() { + return state; + } + + public boolean isAborted() { + return State.ABORTED == this.state; + } + + public boolean isComplete() { + return State.COMPLETED == this.state; + } + + public boolean isInflight() { + return State.INFLIGHT == this.state; + } + + public static String getFileName(String instant, State state) { + return instant + "." + state.name(); + } + + public static List getAllFileNames(String instant) { + return Arrays.stream(State.values()) + .map(state -> getFileName(instant, state)) + .collect(Collectors.toList()); + } + + @Override + public int compareTo(@NotNull CkpMessage o) { + return COMPARATOR.compare(this, o); + } + + /** + * Instant State. + */ + public enum State { + // Inflight instant + INFLIGHT, + // Aborted instant + // An instant can be aborted then be reused again, so it has lower priority + // than COMPLETED + ABORTED, + // Committed instant + COMPLETED + } + + @Override + public String toString() { + return "Ckp{" + "instant='" + instant + '\'' + ", state='" + state + '\'' + '}'; + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java b/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java new file mode 100644 index 0000000000000..3fdba7fd6cf53 --- /dev/null +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java @@ -0,0 +1,226 @@ +/* + * 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.meta; + +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.exception.HoodieException; +import org.apache.hudi.util.StreamerUtil; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The checkpoint metadata for bookkeeping the checkpoint messages. + * + *

Each time the driver starts a new instant, it writes a commit message into the metadata, the write tasks + * then consume the message and unblock the data flushing. + * + *

Why we use the DFS based message queue instead of sending + * the {@link org.apache.flink.runtime.operators.coordination.OperatorEvent} ? + * The write task handles the operator event using the main mailbox executor which has the lowest priority for mails, + * it is also used to process the inputs. When the write task blocks and waits for the operator event to ack the valid instant to write, + * it actually blocks all the subsequent events in the mailbox, the operator event would never be consumed then it causes deadlock. + * + *

The checkpoint metadata is also more lightweight than the active timeline. + * + *

NOTE: should be removed in the future if we have good manner to handle the async notifications from driver. + */ +public class CkpMetadata implements Serializable { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(CkpMetadata.class); + + protected static final int MAX_RETAIN_CKP_NUM = 3; + + // the ckp metadata directory + private static final String CKP_META = "ckp_meta"; + + private final FileSystem fs; + protected final Path path; + + private List messages; + private List instantCache; + + private CkpMetadata(String basePath) { + this(FSUtils.getFs(basePath, StreamerUtil.getHadoopConf()), basePath); + } + + private CkpMetadata(FileSystem fs, String basePath) { + this.fs = fs; + this.path = new Path(ckpMetaPath(basePath)); + } + + public void close() { + this.instantCache = null; + } + + // ------------------------------------------------------------------------- + // WRITE METHODS + // ------------------------------------------------------------------------- + /** + * Initialize the message bus, would clean all the messages and publish the last pending instant. + * + *

This expects to be called by the driver. + */ + public void bootstrap(HoodieTableMetaClient metaClient) throws IOException { + fs.delete(path, true); + fs.mkdirs(path); + metaClient.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction() + .lastInstant().ifPresent(instant -> startInstant(instant.getTimestamp())); + } + + public void startInstant(String instant) { + Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.INFLIGHT)); + try { + fs.createNewFile(path); + } catch (IOException e) { + throw new HoodieException("Exception while adding checkpoint start metadata for instant: " + instant); + } + // cleaning + clean(instant); + } + + private void clean(String newInstant) { + if (this.instantCache == null) { + this.instantCache = new ArrayList<>(); + } + this.instantCache.add(newInstant); + if (instantCache.size() > MAX_RETAIN_CKP_NUM) { + final String instant = instantCache.get(0); + boolean[] error = new boolean[1]; + CkpMessage.getAllFileNames(instant).stream().map(this::fullPath).forEach(path -> { + try { + fs.delete(path, false); + } catch (IOException e) { + error[0] = true; + LOG.warn("Exception while cleaning the checkpoint meta file: " + path); + } + }); + if (!error[0]) { + instantCache.remove(0); + } + } + } + + /** + * Add a checkpoint commit message. + * + * @param instant The committed instant + */ + public void commitInstant(String instant) { + Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.COMPLETED)); + try { + fs.createNewFile(path); + } catch (IOException e) { + throw new HoodieException("Exception while adding checkpoint commit metadata for instant: " + instant); + } + } + + /** + * Add an aborted checkpoint message. + */ + public void abortInstant(String instant) { + Path path = fullPath(CkpMessage.getFileName(instant, CkpMessage.State.ABORTED)); + try { + fs.createNewFile(path); + } catch (IOException e) { + throw new HoodieException("Exception while adding checkpoint abort metadata for instant: " + instant); + } + } + + // ------------------------------------------------------------------------- + // READ METHODS + // ------------------------------------------------------------------------- + + private void load() { + try { + this.messages = scanCkpMetadata(this.path); + } catch (IOException e) { + throw new HoodieException("Exception while scanning the checkpoint meta files under path: " + this.path); + } + } + + @Nullable + public String lastPendingInstant() { + load(); + for (int i = this.messages.size() - 1; i >= 0; i--) { + CkpMessage ckpMsg = this.messages.get(i); + // consider 'aborted' as pending too to reuse the instant + if (!ckpMsg.isComplete()) { + return ckpMsg.getInstant(); + } + } + return null; + } + + public List getMessages() { + load(); + return messages; + } + + public boolean isAborted(String instant) { + ValidationUtils.checkState(this.messages != null, "The checkpoint metadata should #load first"); + return this.messages.stream().anyMatch(ckpMsg -> instant.equals(ckpMsg.getInstant()) && ckpMsg.isAborted()); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + public static CkpMetadata getInstance(String basePath) { + return new CkpMetadata(basePath); + } + + public static CkpMetadata getInstance(FileSystem fs, String basePath) { + return new CkpMetadata(fs, basePath); + } + + protected static String ckpMetaPath(String basePath) { + return basePath + Path.SEPARATOR + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + Path.SEPARATOR + CKP_META; + } + + private Path fullPath(String fileName) { + return new Path(path, fileName); + } + + private List scanCkpMetadata(Path ckpMetaPath) throws IOException { + return Arrays.stream(this.fs.listStatus(ckpMetaPath)).map(CkpMessage::new) + .collect(Collectors.groupingBy(CkpMessage::getInstant)).values().stream() + .map(messages -> messages.stream().reduce((x, y) -> { + // Pick the one with the highest state + if (x.getState().compareTo(y.getState()) >= 0) { + return x; + } + return y; + }).get()) + .sorted().collect(Collectors.toList()); + } +} diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index 3efd1d5612f15..d00eb3e3ec700 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -75,7 +75,6 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.JobConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,7 +92,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes; import static org.apache.hudi.table.format.FormatUtils.getParquetConf; /** @@ -159,7 +157,7 @@ public HoodieTableSource( this.filters = filters == null ? Collections.emptyList() : filters; this.hadoopConf = StreamerUtil.getHadoopConf(); this.metaClient = StreamerUtil.metaClientForReader(conf, hadoopConf); - this.maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new JobConf(this.hadoopConf)); + this.maxCompactionMemoryInBytes = StreamerUtil.getMaxCompactionMemoryInBytes(conf); } @Override diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java index fa404cc2163ec..666dc3a73fb90 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java @@ -32,7 +32,6 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; -import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.avro.Schema; @@ -43,7 +42,7 @@ import org.apache.flink.types.RowKind; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.mapred.JobConf; +import org.apache.hudi.util.StreamerUtil; import java.util.ArrayList; import java.util.Arrays; @@ -190,9 +189,10 @@ public static class BoundedMemoryRecords { public BoundedMemoryRecords( MergeOnReadInputSplit split, Schema logSchema, - Configuration hadoopConf) { + Configuration hadoopConf, + org.apache.flink.configuration.Configuration flinkConf) { this.executor = new BoundedInMemoryExecutor<>( - HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes(new JobConf(hadoopConf)), + StreamerUtil.getMaxCompactionMemoryInBytes(flinkConf), getParallelProducers(), Option.empty(), Function.identity(), diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java index 7fb0b9d5c1ee4..d0d37469349bd 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/CopyOnWriteInputFormat.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.format.cow; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FilePathFilter; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java index ca1408dcb7a5c..e112bcf24e50f 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetSplitReaderUtil.java @@ -22,8 +22,12 @@ import org.apache.hudi.table.format.cow.vector.HeapArrayVector; import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector; import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector; +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.hudi.table.format.cow.vector.reader.ArrayColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.FixedLenBytesColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.Int64TimestampColumnReader; import org.apache.hudi.table.format.cow.vector.reader.MapColumnReader; +import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; import org.apache.hudi.table.format.cow.vector.reader.RowColumnReader; import org.apache.flink.core.fs.Path; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java similarity index 97% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java index 4705b2f633ff5..a2f6d5b0cd74c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetDecimalVector.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/ParquetDecimalVector.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.vector.BytesColumnVector; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/AbstractColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/AbstractColumnReader.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java index efbe914045eff..07416a371715c 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/AbstractColumnReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/AbstractColumnReader.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector.reader; import org.apache.flink.formats.parquet.vector.ParquetDictionary; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java index 256d4c1bbcecd..d94c1e1da4bb6 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ArrayColumnReader.java @@ -18,8 +18,8 @@ package org.apache.hudi.table.format.cow.vector.reader; -import org.apache.hudi.table.format.cow.ParquetDecimalVector; import org.apache.hudi.table.format.cow.vector.HeapArrayVector; +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; import org.apache.flink.table.data.TimestampData; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java similarity index 98% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java index 07a93e19c25c6..61461a728c3b8 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/FixedLenBytesColumnReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/FixedLenBytesColumnReader.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector.reader; import org.apache.flink.table.data.vector.writable.WritableBytesVector; import org.apache.flink.table.data.vector.writable.WritableColumnVector; import org.apache.flink.table.data.vector.writable.WritableIntVector; - import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.io.api.Binary; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/Int64TimestampColumnReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java similarity index 98% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/Int64TimestampColumnReader.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java index 024567c837f1e..555853bda6bd8 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/Int64TimestampColumnReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/Int64TimestampColumnReader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector.reader; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.data.vector.writable.WritableIntVector; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java index 3cb491cfaf575..92f5d1e191771 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/ParquetColumnarRowSplitReader.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/ParquetColumnarRowSplitReader.java @@ -16,7 +16,9 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector.reader; + +import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector; import org.apache.flink.formats.parquet.vector.reader.ColumnReader; import org.apache.flink.table.data.ColumnarRowData; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/RunLengthDecoder.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java similarity index 99% rename from hudi-flink/src/main/java/org/apache/hudi/table/format/cow/RunLengthDecoder.java rename to hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java index 159574714fa19..f13340ceddf92 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/RunLengthDecoder.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/cow/vector/reader/RunLengthDecoder.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hudi.table.format.cow; +package org.apache.hudi.table.format.cow.vector.reader; import org.apache.flink.table.data.vector.writable.WritableColumnVector; import org.apache.flink.table.data.vector.writable.WritableIntVector; diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java index 4404e15eaaccf..8283b5c3cd5b1 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java +++ b/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java @@ -31,7 +31,7 @@ import org.apache.hudi.keygen.KeyGenUtils; import org.apache.hudi.table.format.FilePathUtils; import org.apache.hudi.table.format.FormatUtils; -import org.apache.hudi.table.format.cow.ParquetColumnarRowSplitReader; +import org.apache.hudi.table.format.cow.vector.reader.ParquetColumnarRowSplitReader; import org.apache.hudi.table.format.cow.ParquetSplitReaderUtil; import org.apache.hudi.util.AvroToRowDataConverters; import org.apache.hudi.util.RowDataProjection; @@ -403,7 +403,7 @@ private ClosableIterator getUnMergedLogFileIterator(MergeOnReadInputSpl final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema); final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType()); - final FormatUtils.BoundedMemoryRecords records = new FormatUtils.BoundedMemoryRecords(split, tableSchema, hadoopConf); + final FormatUtils.BoundedMemoryRecords records = new FormatUtils.BoundedMemoryRecords(split, tableSchema, hadoopConf, conf); final Iterator> recordsIterator = records.getRecordsIterator(); return new ClosableIterator() { diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index 45d23f2ff4ea1..55d403dc4db8d 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -488,7 +488,7 @@ public static String getLastPendingInstant(HoodieTableMetaClient metaClient, boo if (reloadTimeline) { metaClient.reloadActiveTimeline(); } - return metaClient.getCommitsTimeline().filterInflights() + return metaClient.getCommitsTimeline().filterPendingExcludingCompaction() .lastInstant() .map(HoodieInstant::getTimestamp) .orElse(null); @@ -509,4 +509,11 @@ public static String getLastCompletedInstant(HoodieTableMetaClient metaClient) { public static boolean haveSuccessfulCommits(HoodieTableMetaClient metaClient) { return !metaClient.getCommitsTimeline().filterCompletedInstants().empty(); } + + /** + * Returns the max compaction memory in bytes with given conf. + */ + public static long getMaxCompactionMemoryInBytes(Configuration conf) { + return conf.getInteger(FlinkOptions.COMPACTION_MAX_MEMORY) * 1024 * 1024; + } } diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java b/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java similarity index 99% rename from hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java rename to hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java index eaa2d6ced67d9..4864696da144c 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java @@ -66,7 +66,7 @@ /** * Integration test for Flink Hoodie stream sink. */ -public class StreamWriteITCase extends TestLogger { +public class ITTestDataStreamWrite extends TestLogger { private static final Map> EXPECTED = new HashMap<>(); private static final Map> EXPECTED_TRANSFORMER = new HashMap<>(); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java index d86602ea95c5b..6266c30523092 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java @@ -18,6 +18,14 @@ package org.apache.hudi.sink; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.FileUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieWriteStat; @@ -30,15 +38,6 @@ import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestConfigurations; import org.apache.hudi.utils.TestUtils; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.util.FileUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -276,6 +275,9 @@ private static WriteMetadataEvent createOperatorEvent( writeStat.setPartitionPath(partitionPath); writeStat.setFileId("fileId123"); writeStat.setPath("path123"); + writeStat.setFileSizeInBytes(123); + writeStat.setTotalWriteBytes(123); + writeStat.setNumWrites(1); writeStatus.setStat(writeStat); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java index 35523a8fb426c..08035aff5a167 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriteCopyOnWrite.java @@ -95,8 +95,8 @@ public void testCheckpointFails() throws Exception { .assertEmptyEvent() .checkpointFails(1) .consume(TestData.DATA_SET_INSERT) - .checkpointThrows(2, - "Timeout(1000ms) while waiting for instant initialize") + //.checkpointThrows(2, + // "Timeout(1000ms) while waiting for instant initialize") // do not send the write event and fails the checkpoint, // behaves like the last checkpoint is successful. .checkpointFails(2) diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index ebe9140adb2ed..e1e86ce32bd82 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -178,10 +178,11 @@ public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exce options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + ""); 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(5); + // insert dataset + tableEnv.executeSql(TestSQL.INSERT_T1).await(); + // update the dataset + tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await(); // Make configuration and setAvroSchema. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -195,8 +196,6 @@ public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exce HoodieFlinkCompactor.AsyncCompactionService asyncCompactionService = new HoodieFlinkCompactor.AsyncCompactionService(cfg, conf, env); asyncCompactionService.start(null); - tableEnv.executeSql(TestSQL.UPDATE_INSERT_T1).await(); - // wait for the asynchronous commit to finish TimeUnit.SECONDS.sleep(5); diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java b/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java new file mode 100644 index 0000000000000..c4eecd7e4941b --- /dev/null +++ b/hudi-flink/src/test/java/org/apache/hudi/sink/meta/TestCkpMetadata.java @@ -0,0 +1,75 @@ +/* + * 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.meta; + +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.util.StreamerUtil; +import org.apache.hudi.utils.TestConfigurations; + +import org.apache.flink.configuration.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.stream.IntStream; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Test cases for {@link CkpMetadata}. + */ +public class TestCkpMetadata { + + private CkpMetadata metadata; + + @TempDir + File tempFile; + + @BeforeEach + public void beforeEach() throws Exception { + String basePath = tempFile.getAbsolutePath(); + FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), StreamerUtil.getHadoopConf()); + + Configuration conf = TestConfigurations.getDefaultConf(basePath); + StreamerUtil.initTableIfNotExists(conf); + + this.metadata = CkpMetadata.getInstance(fs, basePath); + } + + @Test + void testWriteAndReadMessage() { + // write and read 5 committed checkpoints + IntStream.range(0, 3).forEach(i -> metadata.startInstant(i + "")); + + assertThat(metadata.lastPendingInstant(), is("2")); + metadata.commitInstant("2"); + assertThat(metadata.lastPendingInstant(), is("1")); + + // test cleaning + IntStream.range(3, 6).forEach(i -> metadata.startInstant(i + "")); + assertThat(metadata.getMessages().size(), is(3)); + // commit and abort instant does not trigger cleaning + metadata.commitInstant("6"); + metadata.abortInstant("7"); + assertThat(metadata.getMessages().size(), is(5)); + } +} diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java b/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java similarity index 99% rename from hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java rename to hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java index 7c9b0bb6a3cc8..903be90b964be 100644 --- a/hudi-flink/src/test/java/org/apache/hudi/table/HoodieDataSourceITCase.java +++ b/hudi-flink/src/test/java/org/apache/hudi/table/ITTestHoodieDataSource.java @@ -73,7 +73,7 @@ /** * IT cases for Hoodie table source and sink. */ -public class HoodieDataSourceITCase extends AbstractTestBase { +public class ITTestHoodieDataSource extends AbstractTestBase { private TableEnvironment streamTableEnv; private TableEnvironment batchTableEnv; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java index 030e20f2278b4..dfdda9dfc8259 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java @@ -21,21 +21,18 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodiePayloadProps; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.log.LogReaderUtils; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.InputSplitUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -57,6 +54,7 @@ public abstract class AbstractRealtimeRecordReader { private Schema readerSchema; private Schema writerSchema; private Schema hiveSchema; + private HoodieTableMetaClient metaClient; public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) { this.split = split; @@ -65,15 +63,15 @@ public AbstractRealtimeRecordReader(RealtimeSplit split, JobConf job) { LOG.info("columnIds ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); LOG.info("partitioningColumns ==> " + job.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "")); try { - HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jobConf).setBasePath(split.getBasePath()).build(); + metaClient = HoodieTableMetaClient.builder().setConf(jobConf).setBasePath(split.getBasePath()).build(); if (metaClient.getTableConfig().getPreCombineField() != null) { this.payloadProps.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, metaClient.getTableConfig().getPreCombineField()); } this.usesCustomPayload = usesCustomPayload(metaClient); LOG.info("usesCustomPayload ==> " + this.usesCustomPayload); init(); - } catch (IOException e) { - throw new HoodieIOException("Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); + } catch (Exception e) { + throw new HoodieException("Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e); } } @@ -83,19 +81,14 @@ private boolean usesCustomPayload(HoodieTableMetaClient metaClient) { } /** - * Goes through the log files in reverse order and finds the schema from the last available data block. If not, falls + * Gets schema from HoodieTableMetaClient. If not, falls * back to the schema from the latest parquet file. Finally, sets the partition column and projection fields into the * job conf. */ - private void init() throws IOException { - Schema schemaFromLogFile = LogReaderUtils.readLatestSchemaFromLogFiles(split.getBasePath(), split.getDeltaLogFiles(), jobConf); - if (schemaFromLogFile == null) { - writerSchema = InputSplitUtils.getBaseFileSchema((FileSplit)split, jobConf); - LOG.info("Writer Schema From Parquet => " + writerSchema.getFields()); - } else { - writerSchema = schemaFromLogFile; - LOG.info("Writer Schema From Log => " + writerSchema.toString(true)); - } + private void init() throws Exception { + LOG.info("Getting writer schema from table avro schema "); + writerSchema = new TableSchemaResolver(metaClient).getTableAvroSchema(); + // Add partitioning fields to writer schema for resulting row to contain null values for these fields String partitionFields = jobConf.get(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, ""); List partitioningFields = diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/functional/TestHoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/functional/TestHoodieCombineHiveInputFormat.java index ec6ea0a8b3ec3..0a14af2212ac3 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/functional/TestHoodieCombineHiveInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/functional/TestHoodieCombineHiveInputFormat.java @@ -21,12 +21,18 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.io.IOContextMap; import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil; +import org.apache.hudi.common.util.CommitUtils; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat; import org.apache.hudi.hadoop.hive.HoodieCombineRealtimeFileSplit; import org.apache.hudi.hadoop.hive.HoodieCombineRealtimeHiveSplit; @@ -58,6 +64,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.stream.Collectors; @@ -105,7 +112,9 @@ public void multiPartitionReadersRealtimeCombineHoodieInputFormat() throws Excep // Create 3 partitions, each partition holds one parquet file and 1000 records List partitionDirs = InputFormatTestUtil .prepareMultiPartitionedParquetTable(tempDir, schema, 3, numRecords, commitTime, HoodieTableType.MERGE_ON_READ); - InputFormatTestUtil.commit(tempDir, commitTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); TableDesc tblDesc = Utilities.defaultTd; // Set the input format @@ -185,7 +194,9 @@ public void multiLevelPartitionReadersRealtimeCombineHoodieInputFormat() throws final int numRecords = 1000; // Create 3 parquet files with 1000 records each File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); - InputFormatTestUtil.commit(tempDir, commitTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); TableDesc tblDesc = Utilities.defaultTd; // Set the input format @@ -255,7 +266,9 @@ public void testMultiReaderRealtimeCombineHoodieInputFormat() throws Exception { final int numRecords = 1000; // Create 3 parquet files with 1000 records each File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); - InputFormatTestUtil.commit(tempDir, commitTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); String newCommitTime = "101"; // to trigger the bug of HUDI-1772, only update fileid2 @@ -323,7 +336,9 @@ public void testHoodieRealtimeCombineHoodieInputFormat() throws Exception { final int numRecords = 1000; // Create 3 parquet files with 1000 records each File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); - InputFormatTestUtil.commit(tempDir, commitTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); // insert 1000 update records to log file 0 String newCommitTime = "101"; diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java index a6ca32769cf8d..d2c4f1be61d9f 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java @@ -18,11 +18,12 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; + import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.FileSplit; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -33,7 +34,11 @@ import org.mockito.stubbing.Answer; import java.io.DataInput; +import java.io.DataInputStream; import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Collections; @@ -156,4 +161,18 @@ public Void answer(InvocationOnMock invocation) throws Throwable { assertEquals(deltaLogPaths, read.getDeltaLogPaths()); assertEquals(split.toString(), read.toString()); } + + @Test + public void testSerDe(@TempDir java.nio.file.Path tempDir) throws IOException { + final HoodieRealtimeFileSplit original = split; + java.nio.file.Path tempFilePath = tempDir.resolve("tmp.txt"); + try (DataOutputStream out = new DataOutputStream(new FileOutputStream(tempFilePath.toFile()))) { + original.write(out); + } + HoodieRealtimeFileSplit deserialized = new HoodieRealtimeFileSplit(); + try (DataInputStream in = new DataInputStream(new FileInputStream(tempFilePath.toFile()))) { + deserialized.readFields(in); + } + assertEquals(original.toString(), deserialized.toString()); + } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index fc4eb7ce2c042..07a4a0250e5de 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -51,7 +51,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieLogBlock; @@ -60,6 +60,7 @@ import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; @@ -85,7 +86,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -136,16 +136,6 @@ private void setHiveColumnNameProps(List fields, JobConf jobConf, jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveOrderedColumnNames); } - protected Properties getPropertiesForKeyGen() { - Properties properties = new Properties(); - properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false"); - properties.put("hoodie.datasource.write.recordkey.field", "_row_key"); - properties.put("hoodie.datasource.write.partitionpath.field", "partition_path"); - properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key"); - properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path"); - return properties; - } - @ParameterizedTest @MethodSource("testArguments") public void testReader(ExternalSpillableMap.DiskMapType diskMapType, @@ -183,7 +173,10 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, HoodieTableType.MERGE_ON_READ) : InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant, HoodieTableType.MERGE_ON_READ); - FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant); + + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); + FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant, commitMetadata); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -218,7 +211,7 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) fileSlice.addLogFile(writer.getLogFile()); @@ -291,7 +284,9 @@ public void testUnMergedReader() throws Exception { final int secondBatchLastRecordKey = 2 * numRecords - 1; File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime, HoodieTableType.MERGE_ON_READ); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -303,7 +298,7 @@ public void testUnMergedReader() throws Exception { long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime); + FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -371,7 +366,10 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp int numberOfLogRecords = numberOfRecords / 2; File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords, instantTime, HoodieTableType.MERGE_ON_READ); - InputFormatTestUtil.commit(basePath, instantTime); + + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -382,7 +380,9 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - InputFormatTestUtil.deltaCommit(basePath, newCommitTime); + commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); + FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -507,7 +507,9 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa File partitionDir = InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, instantTime, HoodieTableType.MERGE_ON_READ); - InputFormatTestUtil.commit(basePath, instantTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); List firstSchemaFields = schema.getFields(); @@ -529,7 +531,10 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa newCommitTime, "101", 1); logFiles.add(writer.getLogFile()); writer.close(); - InputFormatTestUtil.deltaCommit(basePath, newCommitTime); + + commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -572,6 +577,63 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa reader.close(); } + @Test + public void testSchemaEvolution() throws Exception { + ExternalSpillableMap.DiskMapType diskMapType = ExternalSpillableMap.DiskMapType.BITCASK; + boolean isCompressionEnabled = true; + // initial commit + List logFiles = new ArrayList<>(); + Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()); + HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ); + String instantTime = "100"; + int numberOfRecords = 100; + int numberOfLogRecords = numberOfRecords / 2; + File partitionDir = + InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, + instantTime, HoodieTableType.MERGE_ON_READ); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); + // Add the paths + FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); + List firstSchemaFields = schema.getFields(); + + // 2nd commit w/ evolved schema + Schema evolvedSchema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedCompatibleSchema()); + List secondSchemaFields = evolvedSchema.getFields(); + String newCommitTime = "101"; + File partitionDir1 = + InputFormatTestUtil.prepareSimpleParquetTable(basePath, evolvedSchema, 1, numberOfRecords, + instantTime, HoodieTableType.MERGE_ON_READ,"2017","05","01"); + HoodieCommitMetadata commitMetadata1 = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + evolvedSchema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createCommit(basePath.toString(), newCommitTime, Option.of(commitMetadata1)); + // Add the paths + FileInputFormat.setInputPaths(baseJobConf, partitionDir1.getPath()); + + // create a split with baseFile from 1st commit. + HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( + new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf), + basePath.toUri().toString(), logFiles, newCommitTime, false, Option.empty()); + + // create a RecordReader to be used by HoodieRealtimeRecordReader + RecordReader reader = new MapredParquetInputFormat().getRecordReader( + new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), baseJobConf, null); + JobConf jobConf = new JobConf(baseJobConf); + + // Try to read all the fields passed by the new schema + setHiveColumnNameProps(secondSchemaFields, jobConf, true); + // This time read only the fields which are part of parquet + HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader); + // use reader to read base Parquet File and log file + NullWritable key = recordReader.createKey(); + ArrayWritable value = recordReader.createValue(); + while (recordReader.next(key, value)) { + // keep reading + } + reader.close(); + } + private static Stream testArguments() { // Arg1: ExternalSpillableMap Type, Arg2: isDiskMapCompressionEnabled, Arg3: partitioned return Stream.of( @@ -595,8 +657,7 @@ public void testIncrementalWithOnlylog() throws Exception { final int numRecords = 1000; File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime, HoodieTableType.MERGE_ON_READ); - //FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); - createDeltaCommitFile(basePath, instantTime,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0"); + createDeltaCommitFile(basePath, instantTime,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0", schema.toString()); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -607,7 +668,7 @@ public void testIncrementalWithOnlylog() throws Exception { InputFormatTestUtil.writeDataBlockToLogFile(partitionDir, fs, schema, "fileid0", instantTime, newCommitTime, numRecords, numRecords, 0); writer.close(); - createDeltaCommitFile(basePath, newCommitTime,"2016/05/01", "2016/05/01/.fileid0_100.log.1_1-0-1", "fileid0"); + createDeltaCommitFile(basePath, newCommitTime,"2016/05/01", "2016/05/01/.fileid0_100.log.1_1-0-1", "fileid0", schema.toString()); InputFormatTestUtil.setupIncremental(baseJobConf, "101", 1); @@ -644,8 +705,7 @@ public void testIncrementalWithReplace() throws Exception { String baseInstant = "100"; File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant, HoodieTableType.MERGE_ON_READ); - //FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); - createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0"); + createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0", schema.toString()); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -727,13 +787,17 @@ private void createDeltaCommitFile( String commitNumber, String partitionPath, String filePath, - String fileId) throws IOException { + String fileId, + String schemaStr) throws IOException { List writeStats = new ArrayList<>(); HoodieWriteStat writeStat = createHoodieWriteStat(basePath, commitNumber, partitionPath, filePath, fileId); writeStats.add(writeStat); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); writeStats.forEach(stat -> commitMetadata.addWriteStat(partitionPath, stat)); + if (schemaStr != null) { + commitMetadata.getExtraMetadata().put(HoodieCommitMetadata.SCHEMA_KEY, schemaStr); + } File file = basePath.resolve(".hoodie").resolve(commitNumber + ".deltacommit").toFile(); file.createNewFile(); FileOutputStream fileOutputStream = new FileOutputStream(file); @@ -765,7 +829,9 @@ public void testLogOnlyReader() throws Exception { long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, + schema.toString(), HoodieTimeline.COMMIT_ACTION); + FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); // create a split with new log file(s) fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size)); RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus( @@ -807,7 +873,7 @@ public void testIncrementalWithCompaction() throws Exception { String baseInstant = "100"; File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant, HoodieTableType.MERGE_ON_READ); - createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0"); + createDeltaCommitFile(basePath, baseInstant,"2016/05/01", "2016/05/01/fileid0_1-0-1_100.parquet", "fileid0", schema.toString()); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java index 836ad57121bd5..1185be65c196e 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/testutils/InputFormatTestUtil.java @@ -223,9 +223,14 @@ public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, int numberOfRecords, String commitNumber, HoodieTableType tableType) throws Exception { + return prepareSimpleParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, tableType, "2016","05","01"); + } + + public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles, + int numberOfRecords, String commitNumber, HoodieTableType tableType, String year, String month, String date) throws Exception { HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType, HoodieFileFormat.PARQUET); - java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01")); + java.nio.file.Path partitionPath = basePath.resolve(Paths.get(year, month, date)); setupPartition(basePath, partitionPath); createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber); diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java index 3c7a6034b4f4d..4c0265ce90f64 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java @@ -62,6 +62,7 @@ public abstract class ITTestBase { protected static final String ADHOC_2_CONTAINER = "/adhoc-2"; protected static final String HIVESERVER = "/hiveserver"; protected static final String PRESTO_COORDINATOR = "/presto-coordinator-1"; + protected static final String TRINO_COORDINATOR = "/trino-coordinator-1"; protected static final String HOODIE_WS_ROOT = "/var/hoodie/ws"; protected static final String HOODIE_JAVA_APP = HOODIE_WS_ROOT + "/hudi-spark-datasource/hudi-spark/run_hoodie_app.sh"; protected static final String HOODIE_GENERATE_APP = HOODIE_WS_ROOT + "/hudi-spark-datasource/hudi-spark/run_hoodie_generate_app.sh"; @@ -76,6 +77,7 @@ public abstract class ITTestBase { HOODIE_WS_ROOT + "/docker/hoodie/hadoop/hive_base/target/hoodie-utilities.jar"; protected static final String HIVE_SERVER_JDBC_URL = "jdbc:hive2://hiveserver:10000"; protected static final String PRESTO_COORDINATOR_URL = "presto-coordinator-1:8090"; + protected static final String TRINO_COORDINATOR_URL = "trino-coordinator-1:8091"; protected static final String HADOOP_CONF_DIR = "/etc/hadoop"; // Skip these lines when capturing output from hive @@ -122,6 +124,12 @@ static String getPrestoConsoleCommand(String commandFile) { .append(" -f " + commandFile).toString(); } + static String getTrinoConsoleCommand(String commandFile) { + return new StringBuilder().append("trino --server " + TRINO_COORDINATOR_URL) + .append(" --catalog hive --schema default") + .append(" -f " + commandFile).toString(); + } + @BeforeEach public void init() { String dockerHost = (OVERRIDDEN_DOCKER_HOST != null) ? OVERRIDDEN_DOCKER_HOST : DEFAULT_DOCKER_HOST; @@ -309,6 +317,20 @@ void executePrestoCopyCommand(String fromFile, String remotePath) { .exec(); } + Pair executeTrinoCommandFile(String commandFile) throws Exception { + String trinoCmd = getTrinoConsoleCommand(commandFile); + TestExecStartResultCallback callback = executeCommandStringInDocker(ADHOC_1_CONTAINER, trinoCmd, true); + return Pair.of(callback.getStdout().toString().trim(), callback.getStderr().toString().trim()); + } + + void executeTrinoCopyCommand(String fromFile, String remotePath) { + Container adhocContainer = runningContainers.get(ADHOC_1_CONTAINER); + dockerClient.copyArchiveToContainerCmd(adhocContainer.getId()) + .withHostResource(fromFile) + .withRemotePath(remotePath) + .exec(); + } + private void saveUpLogs() { try { // save up the Hive log files for introspection diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java index b68d06a643298..f142ebd502f15 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.collection.Pair; - import org.apache.hudi.keygen.SimpleKeyGenerator; import org.junit.jupiter.api.AfterEach; @@ -33,24 +32,34 @@ /** * Goes through steps described in https://hudi.apache.org/docker_demo.html - * + *

* To run this as a standalone test in the IDE or command line. First bring up the demo setup using * `docker/setup_demo.sh` and then run the test class as you would do normally. */ public class ITTestHoodieDemo extends ITTestBase { + private static final String TRINO_TABLE_CHECK_FILENAME = "trino-table-check.commands"; + private static final String TRINO_BATCH1_FILENAME = "trino-batch1.commands"; + private static final String TRINO_BATCH2_FILENAME = "trino-batch2-after-compaction.commands"; + private static final String HDFS_DATA_DIR = "/usr/hive/data/input"; private static final String HDFS_BATCH_PATH1 = HDFS_DATA_DIR + "/batch_1.json"; private static final String HDFS_BATCH_PATH2 = HDFS_DATA_DIR + "/batch_2.json"; private static final String HDFS_PRESTO_INPUT_TABLE_CHECK_PATH = HDFS_DATA_DIR + "/presto-table-check.commands"; private static final String HDFS_PRESTO_INPUT_BATCH1_PATH = HDFS_DATA_DIR + "/presto-batch1.commands"; private static final String HDFS_PRESTO_INPUT_BATCH2_PATH = HDFS_DATA_DIR + "/presto-batch2-after-compaction.commands"; + private static final String HDFS_TRINO_INPUT_TABLE_CHECK_PATH = HDFS_DATA_DIR + "/" + TRINO_TABLE_CHECK_FILENAME; + private static final String HDFS_TRINO_INPUT_BATCH1_PATH = HDFS_DATA_DIR + "/" + TRINO_BATCH1_FILENAME; + private static final String HDFS_TRINO_INPUT_BATCH2_PATH = HDFS_DATA_DIR + "/" + TRINO_BATCH2_FILENAME; private static final String INPUT_BATCH_PATH1 = HOODIE_WS_ROOT + "/docker/demo/data/batch_1.json"; private static final String PRESTO_INPUT_TABLE_CHECK_RELATIVE_PATH = "/docker/demo/presto-table-check.commands"; private static final String PRESTO_INPUT_BATCH1_RELATIVE_PATH = "/docker/demo/presto-batch1.commands"; private static final String INPUT_BATCH_PATH2 = HOODIE_WS_ROOT + "/docker/demo/data/batch_2.json"; private static final String PRESTO_INPUT_BATCH2_RELATIVE_PATH = "/docker/demo/presto-batch2-after-compaction.commands"; + private static final String TRINO_INPUT_TABLE_CHECK_RELATIVE_PATH = "/docker/demo/" + TRINO_TABLE_CHECK_FILENAME; + private static final String TRINO_INPUT_BATCH1_RELATIVE_PATH = "/docker/demo/" + TRINO_BATCH1_FILENAME; + private static final String TRINO_INPUT_BATCH2_RELATIVE_PATH = "/docker/demo/" + TRINO_BATCH2_FILENAME; private static final String COW_BASE_PATH = "/user/hive/warehouse/stock_ticks_cow"; private static final String MOR_BASE_PATH = "/user/hive/warehouse/stock_ticks_mor"; @@ -110,12 +119,14 @@ public void testParquetDemo() throws Exception { ingestFirstBatchAndHiveSync(); testHiveAfterFirstBatch(); testPrestoAfterFirstBatch(); + testTrinoAfterFirstBatch(); testSparkSQLAfterFirstBatch(); // batch 2 ingestSecondBatchAndHiveSync(); testHiveAfterSecondBatch(); testPrestoAfterSecondBatch(); + testTrinoAfterSecondBatch(); testSparkSQLAfterSecondBatch(); testIncrementalHiveQueryBeforeCompaction(); testIncrementalSparkSQLQuery(); @@ -125,6 +136,7 @@ public void testParquetDemo() throws Exception { testHiveAfterSecondBatchAfterCompaction(); testPrestoAfterSecondBatchAfterCompaction(); + testTrinoAfterSecondBatchAfterCompaction(); testIncrementalHiveQueryAfterCompaction(); } @@ -133,7 +145,7 @@ public void testParquetDemo() throws Exception { public void testHFileDemo() throws Exception { baseFileFormat = HoodieFileFormat.HFILE; - // TODO: Preseto and SparkSQL support for HFile format + // TODO: Presto, Trino and SparkSQL support for HFile format setupDemo(); @@ -141,12 +153,14 @@ public void testHFileDemo() throws Exception { ingestFirstBatchAndHiveSync(); testHiveAfterFirstBatch(); //testPrestoAfterFirstBatch(); + //testTrinoAfterFirstBatch(); //testSparkSQLAfterFirstBatch(); // batch 2 ingestSecondBatchAndHiveSync(); testHiveAfterSecondBatch(); //testPrestoAfterSecondBatch(); + //testTrinoAfterSecondBatch(); //testSparkSQLAfterSecondBatch(); testIncrementalHiveQueryBeforeCompaction(); //testIncrementalSparkSQLQuery(); @@ -155,6 +169,7 @@ public void testHFileDemo() throws Exception { scheduleAndRunCompaction(); testHiveAfterSecondBatchAfterCompaction(); //testPrestoAfterSecondBatchAfterCompaction(); + //testTrinoAfterSecondBatchAfterCompaction(); //testIncrementalHiveQueryAfterCompaction(); } @@ -162,7 +177,8 @@ private void setupDemo() throws Exception { List cmds = CollectionUtils.createImmutableList("hdfs dfsadmin -safemode wait", "hdfs dfs -mkdir -p " + HDFS_DATA_DIR, "hdfs dfs -copyFromLocal -f " + INPUT_BATCH_PATH1 + " " + HDFS_BATCH_PATH1, - "/bin/bash " + DEMO_CONTAINER_SCRIPT); + "/bin/bash " + DEMO_CONTAINER_SCRIPT, + "mkdir -p " + HDFS_DATA_DIR); executeCommandStringsInDocker(ADHOC_1_CONTAINER, cmds); @@ -174,6 +190,10 @@ private void setupDemo() throws Exception { executePrestoCopyCommand(System.getProperty("user.dir") + "/.." + PRESTO_INPUT_TABLE_CHECK_RELATIVE_PATH, HDFS_DATA_DIR); executePrestoCopyCommand(System.getProperty("user.dir") + "/.." + PRESTO_INPUT_BATCH1_RELATIVE_PATH, HDFS_DATA_DIR); executePrestoCopyCommand(System.getProperty("user.dir") + "/.." + PRESTO_INPUT_BATCH2_RELATIVE_PATH, HDFS_DATA_DIR); + + executeTrinoCopyCommand(System.getProperty("user.dir") + "/.." + TRINO_INPUT_TABLE_CHECK_RELATIVE_PATH, HDFS_DATA_DIR); + executeTrinoCopyCommand(System.getProperty("user.dir") + "/.." + TRINO_INPUT_BATCH1_RELATIVE_PATH, HDFS_DATA_DIR); + executeTrinoCopyCommand(System.getProperty("user.dir") + "/.." + TRINO_INPUT_BATCH2_RELATIVE_PATH, HDFS_DATA_DIR); } private void ingestFirstBatchAndHiveSync() throws Exception { @@ -335,6 +355,20 @@ private void testPrestoAfterFirstBatch() throws Exception { "\"GOOG\",\"2018-08-31 10:29:00\",\"3391\",\"1230.1899\",\"1230.085\"", 2); } + private void testTrinoAfterFirstBatch() throws Exception { + Pair stdOutErrPair = executeTrinoCommandFile(HDFS_TRINO_INPUT_TABLE_CHECK_PATH); + assertStdOutContains(stdOutErrPair, "stock_ticks_cow", 2); + assertStdOutContains(stdOutErrPair, "stock_ticks_mor", 4); + + stdOutErrPair = executeTrinoCommandFile(HDFS_TRINO_INPUT_BATCH1_PATH); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:29:00\"", 4); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 09:59:00\",\"6330\",\"1230.5\",\"1230.02\"", 2); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:29:00\",\"3391\",\"1230.1899\",\"1230.085\"", 2); + } + private void testHiveAfterSecondBatch() throws Exception { Pair stdOutErrPair = executeHiveCommandFile(HIVE_BATCH1_COMMANDS); assertStdOutContains(stdOutErrPair, "| symbol | _c1 |\n+---------+----------------------+\n" @@ -361,7 +395,21 @@ private void testPrestoAfterSecondBatch() throws Exception { assertStdOutContains(stdOutErrPair, "\"GOOG\",\"2018-08-31 10:59:00\"", 2); assertStdOutContains(stdOutErrPair, - "\"GOOG\",\"2018-08-31 09:59:00\",\"6330\",\"1230.5\",\"1230.02\"",2); + "\"GOOG\",\"2018-08-31 09:59:00\",\"6330\",\"1230.5\",\"1230.02\"", 2); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:29:00\",\"3391\",\"1230.1899\",\"1230.085\""); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:59:00\",\"9021\",\"1227.1993\",\"1227.215\""); + } + + private void testTrinoAfterSecondBatch() throws Exception { + Pair stdOutErrPair = executeTrinoCommandFile(HDFS_TRINO_INPUT_BATCH1_PATH); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:29:00\"", 2); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:59:00\"", 2); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 09:59:00\",\"6330\",\"1230.5\",\"1230.02\"", 2); assertStdOutContains(stdOutErrPair, "\"GOOG\",\"2018-08-31 10:29:00\",\"3391\",\"1230.1899\",\"1230.085\""); assertStdOutContains(stdOutErrPair, @@ -390,6 +438,16 @@ private void testPrestoAfterSecondBatchAfterCompaction() throws Exception { "\"GOOG\",\"2018-08-31 10:59:00\",\"9021\",\"1227.1993\",\"1227.215\""); } + private void testTrinoAfterSecondBatchAfterCompaction() throws Exception { + Pair stdOutErrPair = executeTrinoCommandFile(HDFS_TRINO_INPUT_BATCH2_PATH); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:59:00\"", 2); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 09:59:00\",\"6330\",\"1230.5\",\"1230.02\""); + assertStdOutContains(stdOutErrPair, + "\"GOOG\",\"2018-08-31 10:59:00\",\"9021\",\"1227.1993\",\"1227.215\""); + } + private void testSparkSQLAfterSecondBatch() throws Exception { Pair stdOutErrPair = executeSparkSQLCommand(SPARKSQL_BATCH2_COMMANDS, true); assertStdOutContains(stdOutErrPair, diff --git a/hudi-spark-datasource/README.md b/hudi-spark-datasource/README.md index 50a88ac8d2403..c423a8b2ce9b1 100644 --- a/hudi-spark-datasource/README.md +++ b/hudi-spark-datasource/README.md @@ -36,3 +36,21 @@ file that supports spark sql on spark 2.x version. has no class since hudi only supports spark 2.4.4 version, and it acts as the placeholder when packaging hudi-spark-bundle module. * hudi-spark3-common is the module that contains the code that would be reused between spark3.x versions. * hudi-spark-common is the module that contains the code that would be reused between spark2.x and spark3.x versions. + +## Description of Time Travel +* `HoodieSpark3_2ExtendedSqlAstBuilder` have comments in the spark3.2's code fork from `org.apache.spark.sql.catalyst.parser.AstBuilder`, and additional `withTimeTravel` method. +* `SqlBase.g4` have comments in the code forked from spark3.2's parser, and add SparkSQL Syntax `TIMESTAMP AS OF` and `VERSION AS OF`. + +### Time Travel Support Spark Version: + +| version | support | +| ------ | ------- | +| 2.4.x | No | +| 3.0.x | No | +| 3.1.2 | No | +| 3.2.0 | Yes | + +### About upgrading Time Travel: +Spark3.3 support time travel syntax link [SPARK-37219](https://issues.apache.org/jira/browse/SPARK-37219). +Once Spark 3.3 released. The files in the following list will be removed: +* hudi-spark3's `HoodieSpark3_2ExtendedSqlAstBuilder.scala`、`HoodieSpark3_2ExtendedSqlParser.scala`、`TimeTravelRelation.scala`、`SqlBase.g4`、`HoodieSqlBase.g4` diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index bad6c2d7219e1..634389b1e68b1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -32,7 +32,6 @@ import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.TablePathUtils; -import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -177,8 +176,6 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key())); boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE().key()) .equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL()); - boolean asyncClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key())); - boolean inlineClusteringEnabled = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INLINE_CLUSTERING_ENABLE().key())); // insert/bulk-insert combining to be true, if filtering for duplicates boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS().key())); HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() @@ -191,9 +188,6 @@ public static HoodieWriteConfig createHoodieConfig(String schemaStr, String base .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key())) .withInlineCompaction(inlineCompact).build()) - .withClusteringConfig(HoodieClusteringConfig.newBuilder() - .withInlineClustering(inlineClusteringEnabled) - .withAsyncClustering(asyncClusteringEnabled).build()) .withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(parameters.get(DataSourceWriteOptions.PRECOMBINE_FIELD().key())) .build()) // override above with Hoodie configs specified as options. @@ -322,6 +316,8 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) { hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION()); } + hiveSyncConfig.syncComment = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT().key(), + DataSourceWriteOptions.HIVE_SYNC_COMMENT().defaultValue())); return hiveSyncConfig; } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala index 8e94805328c69..473bb2e2445bd 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/BaseFileOnlyViewRelation.scala @@ -18,63 +18,83 @@ package org.apache.hudi +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path - +import org.apache.hudi.HoodieBaseRelation.createBaseFileReader import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.hadoop.HoodieROTablePathFilter - import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, SQLContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} -import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.{BaseRelation, Filter} -import org.apache.spark.sql.types.{BooleanType, StructType} +import org.apache.spark.sql.types.StructType /** - * The implement of [[BaseRelation]], which is used to respond to query that only touches the base files(Parquet), - * like query COW tables in Snapshot-Query and Read_Optimized mode and MOR tables in Read_Optimized mode. + * [[BaseRelation]] implementation only reading Base files of Hudi tables, essentially supporting following querying + * modes: + *

    + *
  • For COW tables: Snapshot
  • + *
  • For MOR tables: Read-optimized
  • + *
+ * + * NOTE: The reason this Relation is used in liue of Spark's default [[HadoopFsRelation]] is primarily due to the + * fact that it injects real partition's path as the value of the partition field, which Hudi ultimately persists + * as part of the record payload. In some cases, however, partition path might not necessarily be equal to the + * verbatim value of the partition path field (when custom [[KeyGenerator]] is used) therefore leading to incorrect + * partition field values being written */ -class BaseFileOnlyViewRelation( - sqlContext: SQLContext, - metaClient: HoodieTableMetaClient, - optParams: Map[String, String], - userSchema: Option[StructType], - globPaths: Seq[Path] - ) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) with SparkAdapterSupport { - - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { - sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false") - - val filterExpressions = HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema) - .getOrElse(Literal(true, BooleanType)) - val (partitionFilters, dataFilters) = { - val splited = filters.map { filter => - HoodieDataSourceHelper.splitPartitionAndDataPredicates( - sparkSession, filterExpressions, partitionColumns) - } - (splited.flatMap(_._1), splited.flatMap(_._2)) - } - val partitionFiles = getPartitionFiles(partitionFilters, dataFilters) +class BaseFileOnlyViewRelation(sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, + optParams: Map[String, String], + userSchema: Option[StructType], + globPaths: Seq[Path]) + extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) with SparkAdapterSupport { - val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes - val filePartitions = sparkAdapter.getFilePartitions(sparkSession, partitionFiles, maxSplitBytes) + private val fileIndex = HoodieFileIndex(sparkSession, metaClient, userSchema, optParams, + FileStatusCache.getOrCreate(sqlContext.sparkSession)) + + override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { + // NOTE: In case list of requested columns doesn't contain the Primary Key one, we + // have to add it explicitly so that + // - Merging could be performed correctly + // - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]], + // Spark still fetches all the rows to execute the query correctly + // + // It's okay to return columns that have not been requested by the caller, as those nevertheless will be + // filtered out upstream + val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns) + + val (requiredAvroSchema, requiredStructSchema) = + HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns) + + val filterExpressions = convertToExpressions(filters) + val (partitionFilters, dataFilters) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates( + sparkSession, filterExpressions, partitionColumns) - val requiredSchemaParquetReader = HoodieDataSourceHelper.buildHoodieParquetReader( - sparkSession = sparkSession, - dataSchema = tableStructSchema, - partitionSchema = StructType(Nil), - requiredSchema = tableStructSchema, + val filePartitions = getPartitions(partitionFilters, dataFilters) + + val partitionSchema = StructType(Nil) + val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchema.toString) + val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString) + + val baseFileReader = createBaseFileReader( + spark = sparkSession, + partitionSchema = partitionSchema, + tableSchema = tableSchema, + requiredSchema = requiredSchema, filters = filters, options = optParams, - hadoopConf = sparkSession.sessionState.newHadoopConf() + // NOTE: We have to fork the Hadoop Config here as Spark will be modifying it + // to configure Parquet reader appropriately + hadoopConf = new Configuration(conf) ) - new HoodieFileScanRDD(sparkSession, requiredColumns, tableStructSchema, - requiredSchemaParquetReader, filePartitions) + new HoodieFileScanRDD(sparkSession, baseFileReader, filePartitions) } - private def getPartitionFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionedFile] = { + private def getPartitions(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[FilePartition] = { val partitionDirectories = if (globPaths.isEmpty) { val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, userSchema, optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession)) @@ -89,18 +109,33 @@ class BaseFileOnlyViewRelation( inMemoryFileIndex.listFiles(partitionFilters, dataFilters) } - val partitionFiles = partitionDirectories.flatMap { partition => + val partitions = partitionDirectories.flatMap { partition => partition.files.flatMap { file => + // TODO move to adapter + // TODO fix, currently assuming parquet as underlying format HoodieDataSourceHelper.splitFiles( sparkSession = sparkSession, file = file, - partitionValues = partition.values + // TODO clarify why this is required + partitionValues = InternalRow.empty ) } } - partitionFiles.map{ f => - PartitionedFile(InternalRow.empty, f.filePath, f.start, f.length) + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + + sparkAdapter.getFilePartitions(sparkSession, partitions, maxSplitBytes) + } + + private def convertToExpressions(filters: Array[Filter]): Array[Expression] = { + val catalystExpressions = HoodieSparkUtils.convertToCatalystExpressions(filters, tableStructSchema) + + val failedExprs = catalystExpressions.zipWithIndex.filter { case (opt, _) => opt.isEmpty } + if (failedExprs.nonEmpty) { + val failedFilters = failedExprs.map(p => filters(p._2)) + logWarning(s"Failed to convert Filters into Catalyst expressions (${failedFilters.map(_.toString)})") } + + catalystExpressions.filter(_.isDefined).map(_.get).toArray } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 8a98657f242e2..530e435783696 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -555,6 +555,11 @@ object DataSourceWriteOptions { .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'") + val HIVE_SYNC_COMMENT: ConfigProperty[String] = ConfigProperty + .key("hoodie.datasource.hive_sync.sync_comment") + .defaultValue("false") + .withDocumentation("Whether to sync the table column comments while syncing the table.") + // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty .key("hoodie.datasource.compaction.async.enable") diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index 1e2946dd26e88..e07b316d48db3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -22,38 +22,70 @@ import org.apache.avro.generic.GenericRecord import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hbase.io.hfile.CacheConfig +import org.apache.hadoop.mapred.JobConf +import org.apache.hudi.HoodieBaseRelation.isMetadataTable import org.apache.hudi.common.config.SerializableConfiguration import org.apache.hudi.common.fs.FSUtils -import org.apache.hudi.common.model.HoodieFileFormat +import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.StringUtils import org.apache.hudi.io.storage.HoodieHFileReader -import org.apache.hudi.metadata.HoodieTableMetadata +import org.apache.hudi.metadata.{HoodieMetadataPayload, HoodieTableMetadata} import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.sql.avro.SchemaConverters import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{SQLContext, SparkSession} +import org.apache.spark.sql.{Row, SQLContext, SparkSession} import scala.collection.JavaConverters._ import scala.util.Try case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String) +case class HoodieTableState(recordKeyField: String, + preCombineFieldOpt: Option[String]) + /** * Hoodie BaseRelation which extends [[PrunedFilteredScan]]. */ -abstract class HoodieBaseRelation( - val sqlContext: SQLContext, - metaClient: HoodieTableMetaClient, - optParams: Map[String, String], - userSchema: Option[StructType]) - extends BaseRelation with PrunedFilteredScan with Logging{ +abstract class HoodieBaseRelation(val sqlContext: SQLContext, + metaClient: HoodieTableMetaClient, + optParams: Map[String, String], + userSchema: Option[StructType]) + extends BaseRelation with PrunedFilteredScan with Logging { protected val sparkSession: SparkSession = sqlContext.sparkSession + protected lazy val conf: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + protected lazy val jobConf = new JobConf(conf) + + // If meta fields are enabled, always prefer key from the meta field as opposed to user-specified one + // NOTE: This is historical behavior which is preserved as is + protected lazy val recordKeyField: String = + if (metaClient.getTableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD + else metaClient.getTableConfig.getRecordKeyFieldProp + + protected lazy val preCombineFieldOpt: Option[String] = getPrecombineFieldProperty + + /** + * @VisibleInTests + */ + lazy val mandatoryColumns: Seq[String] = { + if (isMetadataTable(metaClient)) { + Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE) + } else { + Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) + } + } + + protected lazy val specifiedQueryInstant: Option[String] = + optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key) + .map(HoodieSqlCommonUtils.formatQueryInstant) + protected lazy val tableAvroSchema: Schema = { val schemaUtil = new TableSchemaResolver(metaClient) Try(schemaUtil.getTableAvroSchema).getOrElse( @@ -81,6 +113,34 @@ abstract class HoodieBaseRelation( } override def schema: StructType = tableStructSchema + + /** + * This method controls whether relation will be producing + *
    + *
  • [[Row]], when it's being equal to true
  • + *
  • [[InternalRow]], when it's being equal to false
  • + *
+ * + * Returning [[InternalRow]] directly enables us to save on needless ser/de loop from [[InternalRow]] (being + * produced by file-reader) to [[Row]] and back + */ + override final def needConversion: Boolean = false + + /** + * NOTE: DO NOT OVERRIDE THIS METHOD + */ + override final def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + // Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]] + // Please check [[needConversion]] scala-doc for more details + doBuildScan(requiredColumns, filters).asInstanceOf[RDD[Row]] + } + + protected def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] + + protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { + val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) + requestedColumns ++ missing + } } object HoodieBaseRelation { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala new file mode 100644 index 0000000000000..58c33248234c2 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieCLIUtils.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi + +import org.apache.hudi.client.SparkRDDWriteClient +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.withSparkConf + +import scala.collection.JavaConverters.mapAsJavaMapConverter +import scala.collection.immutable.Map + +object HoodieCLIUtils { + + def createHoodieClientFromPath(sparkSession: SparkSession, + basePath: String, + conf: Map[String, String]): SparkRDDWriteClient[_] = { + val metaClient = HoodieTableMetaClient.builder().setBasePath(basePath) + .setConf(sparkSession.sessionState.newHadoopConf()).build() + val schemaUtil = new TableSchemaResolver(metaClient) + val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString + val finalParameters = HoodieWriterUtils.parametersWithWriteDefaults( + withSparkConf(sparkSession, Map.empty)( + conf + (DataSourceWriteOptions.TABLE_TYPE.key() -> metaClient.getTableType.name())) + ) + + val jsc = new JavaSparkContext(sparkSession.sparkContext) + DataSourceUtils.createHoodieClient(jsc, schemaStr, basePath, + metaClient.getTableConfig.getTableName, finalParameters.asJava) + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala index fb12549f620bd..40299cfdcd6f1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieDataSourceHelper.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SpecificInternalRow, SubqueryExpression, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{PredicateHelper, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.sources.Filter @@ -33,43 +33,6 @@ import scala.collection.JavaConverters._ object HoodieDataSourceHelper extends PredicateHelper { - /** - * Partition the given condition into two sequence of conjunctive predicates: - * - predicates that can be evaluated using metadata only. - * - other predicates. - */ - def splitPartitionAndDataPredicates( - spark: SparkSession, - condition: Expression, - partitionColumns: Seq[String]): (Seq[Expression], Seq[Expression]) = { - splitConjunctivePredicates(condition).partition( - isPredicateMetadataOnly(spark, _, partitionColumns)) - } - - /** - * Check if condition can be evaluated using only metadata. In Delta, this means the condition - * only references partition columns and involves no subquery. - */ - def isPredicateMetadataOnly( - spark: SparkSession, - condition: Expression, - partitionColumns: Seq[String]): Boolean = { - isPredicatePartitionColumnsOnly(spark, condition, partitionColumns) && - !SubqueryExpression.hasSubquery(condition) - } - - /** - * Does the predicate only contains partition columns? - */ - def isPredicatePartitionColumnsOnly( - spark: SparkSession, - condition: Expression, - partitionColumns: Seq[String]): Boolean = { - val nameEquality = spark.sessionState.analyzer.resolver - condition.references.forall { r => - partitionColumns.exists(nameEquality(r.name, _)) - } - } /** * Wrapper `buildReaderWithPartitionValues` of [[ParquetFileFormat]] diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala index 9f2d7d9e0380a..7e8f62bd2500a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileScanRDD.scala @@ -18,56 +18,37 @@ package org.apache.hudi -import org.apache.spark.{Partition, TaskContext} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, SchemaColumnConvertNotSupportedException} -import org.apache.spark.sql.types.StructType +import org.apache.spark.{Partition, TaskContext} /** - * Similar to [[org.apache.spark.sql.execution.datasources.FileScanRDD]]. - * - * This class will extract the fields needed according to [[requiredColumns]] and - * return iterator of [[org.apache.spark.sql.Row]] directly. + * TODO eval if we actually need it */ -class HoodieFileScanRDD( - @transient private val sparkSession: SparkSession, - requiredColumns: Array[String], - schema: StructType, - readFunction: PartitionedFile => Iterator[InternalRow], - @transient val filePartitions: Seq[FilePartition]) - extends RDD[Row](sparkSession.sparkContext, Nil) { - - private val requiredSchema = { - val nameToStructField = schema.map(field => (field.name, field)).toMap - StructType(requiredColumns.map(nameToStructField)) - } - - private val requiredFieldPos = HoodieSparkUtils.collectFieldIndexes(requiredSchema, schema) - - override def compute(split: Partition, context: TaskContext): Iterator[Row] = { - val iterator = new Iterator[Object] with AutoCloseable { +class HoodieFileScanRDD(@transient private val sparkSession: SparkSession, + readFunction: PartitionedFile => Iterator[InternalRow], + @transient fileSplits: Seq[FilePartition]) + extends HoodieUnsafeRDD(sparkSession.sparkContext) { + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val iterator = new Iterator[InternalRow] with AutoCloseable { private[this] val files = split.asInstanceOf[FilePartition].files.toIterator - private[this] var currentFile: PartitionedFile = null - private[this] var currentIterator: Iterator[Object] = null + private[this] var currentFile: PartitionedFile = _ + private[this] var currentIterator: Iterator[InternalRow] = _ override def hasNext: Boolean = { (currentIterator != null && currentIterator.hasNext) || nextIterator() } - def next(): Object = { - currentIterator.next() - } + def next(): InternalRow = currentIterator.next() /** Advances to the next file. Returns true if a new non-empty iterator is available. */ private def nextIterator(): Boolean = { if (files.hasNext) { - currentFile = files.next() - logInfo(s"Reading File $currentFile") + currentFile = files.next() currentIterator = readFunction(currentFile) try { @@ -93,17 +74,8 @@ class HoodieFileScanRDD( // Register an on-task-completion callback to close the input stream. context.addTaskCompletionListener[Unit](_ => iterator.close()) - // extract required columns from row - val iterAfterExtract = HoodieDataSourceHelper.extractRequiredSchema( - iterator.asInstanceOf[Iterator[InternalRow]], - requiredSchema, - requiredFieldPos) - - // convert InternalRow to Row and return - val converter = CatalystTypeConverters.createToScalaConverter(requiredSchema) - iterAfterExtract.map(converter(_).asInstanceOf[Row]) + iterator.asInstanceOf[Iterator[InternalRow]] } - override protected def getPartitions: Array[Partition] = filePartitions.toArray - + override protected def getPartitions: Array[Partition] = fileSplits.toArray } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala index 96fe47e0219d4..3a518da32b3dd 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala @@ -19,12 +19,13 @@ package org.apache.hudi import org.apache.avro.Schema -import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder} +import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder, IndexedRecord} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieDataSourceHelper._ import org.apache.hudi.HoodieMergeOnReadRDD.resolveAvroSchemaNullability import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath +import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.engine.HoodieLocalEngineContext import org.apache.hudi.common.fs.FSUtils @@ -35,7 +36,6 @@ import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.config.HoodieRealtimeConfig import org.apache.hudi.metadata.HoodieTableMetadata.getDataTableBasePathFromMetadataTable import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadata} -import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection import org.apache.spark.sql.execution.datasources.PartitionedFile @@ -53,10 +53,11 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, @transient config: Configuration, fullSchemaFileReader: PartitionedFile => Iterator[InternalRow], requiredSchemaFileReader: PartitionedFile => Iterator[InternalRow], - tableState: HoodieMergeOnReadTableState, + tableState: HoodieTableState, tableSchema: HoodieTableSchema, - requiredSchema: HoodieTableSchema) - extends RDD[InternalRow](sc, Nil) { + requiredSchema: HoodieTableSchema, + @transient fileSplits: List[HoodieMergeOnReadFileSplit]) + extends HoodieUnsafeRDD(sc) { private val confBroadcast = sc.broadcast(new SerializableWritable(config)) private val recordKeyField = tableState.recordKeyField @@ -97,12 +98,8 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, iter } - override protected def getPartitions: Array[Partition] = { - tableState - .hoodieRealtimeFileSplits - .zipWithIndex - .map(file => HoodieMergeOnReadPartition(file._2, file._1)).toArray - } + override protected def getPartitions: Array[Partition] = + fileSplits.zipWithIndex.map(file => HoodieMergeOnReadPartition(file._2, file._1)).toArray private def getConfig: Configuration = { val conf = confBroadcast.value.value @@ -309,10 +306,15 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext, } } - private def mergeRowWithLog(curRow: InternalRow, curKey: String) = { + private def mergeRowWithLog(curRow: InternalRow, curKey: String) : org.apache.hudi.common.util.Option[IndexedRecord] = { val historyAvroRecord = serializer.serialize(curRow).asInstanceOf[GenericRecord] - logRecords.get(curKey).getData + val mergedRec = logRecords.get(curKey).getData .combineAndGetUpdateValue(historyAvroRecord, tableAvroSchema, payloadProps) + if (mergedRec.isPresent && mergedRec.get().getSchema != tableAvroSchema) { + org.apache.hudi.common.util.Option.of(HoodieAvroUtils.rewriteRecord(mergedRec.get().asInstanceOf[GenericRecord], tableAvroSchema).asInstanceOf[IndexedRecord]) + } else { + mergedRec + } } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 6b6ddc38e3039..8ba0db907b370 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -395,7 +395,8 @@ object HoodieSparkSqlWriter { val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters) val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD) val keyGenProp = hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME) - val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean + val populateMetaFields = java.lang.Boolean.parseBoolean((parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), + String.valueOf(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())))) val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT) HoodieTableMetaClient.withPropertyBuilder() @@ -447,8 +448,8 @@ object HoodieSparkSqlWriter { instantTime: String, partitionColumns: String): (Boolean, common.util.Option[String]) = { val sparkContext = sqlContext.sparkContext - val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), - HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean + val populateMetaFields = java.lang.Boolean.parseBoolean((parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), + String.valueOf(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())))) val dropPartitionColumns = parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean // register classes & schemas @@ -600,6 +601,7 @@ object HoodieSparkSqlWriter { hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES) hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES) hiveSyncConfig.sparkVersion = SPARK_VERSION + hiveSyncConfig.syncComment = hoodieConfig.getStringOrDefault(HIVE_SYNC_COMMENT).toBoolean hiveSyncConfig } @@ -732,8 +734,7 @@ object HoodieSparkSqlWriter { private def isAsyncClusteringEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]], parameters: Map[String, String]): Boolean = { log.info(s"Config.asyncClusteringEnabled ? ${client.getConfig.isAsyncClusteringEnabled}") - asyncClusteringTriggerFnDefined && client.getConfig.isAsyncClusteringEnabled && - parameters.get(ASYNC_CLUSTERING_ENABLE.key).exists(r => r.toBoolean) + asyncClusteringTriggerFnDefined && client.getConfig.isAsyncClusteringEnabled } private def getHoodieTableConfig(sparkContext: SparkContext, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala new file mode 100644 index 0000000000000..3f95746a54669 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieUnsafeRDD.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.{Partition, SparkContext, TaskContext} + +/** + * !!! PLEASE READ CAREFULLY !!! + * + * Base class for all of the custom low-overhead RDD implementations for Hudi. + * + * To keep memory allocation footprint as low as possible, each inheritor of this RDD base class + * + *
+ *   1. Does NOT deserialize from [[InternalRow]] to [[Row]] (therefore only providing access to
+ *   Catalyst internal representations (often mutable) of the read row)
+ *
+ *   2. DOES NOT COPY UNDERLYING ROW OUT OF THE BOX, meaning that
+ *
+ *      a) access to this RDD is NOT thread-safe
+ *
+ *      b) iterating over it reference to a _mutable_ underlying instance (of [[InternalRow]]) is
+ *      returned, entailing that after [[Iterator#next()]] is invoked on the provided iterator,
+ *      previous reference becomes **invalid**. Therefore, you will have to copy underlying mutable
+ *      instance of [[InternalRow]] if you plan to access it after [[Iterator#next()]] is invoked (filling
+ *      it with the next row's payload)
+ *
+ *      c) due to item b) above, no operation other than the iteration will produce meaningful
+ *      results on it and will likely fail [1]
+ * 
+ * + * [1] For example, [[RDD#collect]] method on this implementation would not work correctly, as it's + * simply using Scala's default [[Iterator#toArray]] method which will simply concat all the references onto + * the same underlying mutable object into [[Array]]. Instead each individual [[InternalRow]] _has to be copied_, + * before concatenating into the final output. Please refer to [[HoodieRDDUtils#collect]] for more details. + * + * NOTE: It enforces, for ex, that all of the RDDs implement [[compute]] method returning + * [[InternalRow]] to avoid superfluous ser/de + */ +abstract class HoodieUnsafeRDD(@transient sc: SparkContext) + extends RDD[InternalRow](sc, Nil) { + + def compute(split: Partition, context: TaskContext): Iterator[InternalRow] + + override final def collect(): Array[InternalRow] = + throw new UnsupportedOperationException( + "This method will not function correctly, please refer to scala-doc for HoodieUnsafeRDD" + ) +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index b9d18c68d3d60..8308e3b7ee8ad 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -19,7 +19,6 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{GlobPattern, Path} -import org.apache.hadoop.mapred.JobConf import org.apache.hudi.HoodieBaseRelation.createBaseFileReader import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.table.HoodieTableMetaClient @@ -28,11 +27,11 @@ import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.{getCommitMetadata, getWritePartitionPaths, listAffectedFilesForCommits} import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{Row, SQLContext} import scala.collection.JavaConversions._ @@ -47,9 +46,6 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, val metaClient: HoodieTableMetaClient) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { - private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - private val jobConf = new JobConf(conf) - private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() if (commitTimeline.empty()) { throw new HoodieException("No instants to incrementally pull") @@ -77,8 +73,6 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, private val fileIndex = if (commitsToReturn.isEmpty) List() else buildFileIndex() - private val preCombineFieldOpt = getPrecombineFieldProperty - // Record filters making sure that only records w/in the requested bounds are being fetched as part of the // scan collected by this relation private lazy val incrementalSpanRecordsFilters: Seq[Filter] = { @@ -88,18 +82,16 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, Seq(isNotNullFilter, largerThanFilter, lessThanFilter) } - private lazy val mandatoryColumns = { + override lazy val mandatoryColumns: Seq[String] = { // NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in // cases when no columns are requested to be fetched (for ex, when using {@code count()} API) Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) } - override def needConversion: Boolean = false - - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { if (fileIndex.isEmpty) { - sqlContext.sparkContext.emptyRDD[Row] + sqlContext.sparkContext.emptyRDD[InternalRow] } else { logDebug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}") logDebug(s"buildScan filters = ${filters.mkString(",")}") @@ -148,20 +140,20 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, hadoopConf = new Configuration(conf) ) - val hoodieTableState = HoodieMergeOnReadTableState(fileIndex, HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt) + val hoodieTableState = HoodieTableState(HoodieRecord.RECORD_KEY_METADATA_FIELD, preCombineFieldOpt) // TODO implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately // filtered, since file-reader might not be capable to perform filtering - val rdd = new HoodieMergeOnReadRDD( + new HoodieMergeOnReadRDD( sqlContext.sparkContext, jobConf, fullSchemaParquetReader, requiredSchemaParquetReader, hoodieTableState, tableSchema, - requiredSchema + requiredSchema, + fileIndex ) - rdd.asInstanceOf[RDD[Row]] } } @@ -225,9 +217,4 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType) }) } - - private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { - val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) - requestedColumns ++ missing - } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala index 7c1a3540c814e..6156054b4f45b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadSnapshotRelation.scala @@ -20,22 +20,19 @@ package org.apache.hudi import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.JobConf -import org.apache.hudi.HoodieBaseRelation.{createBaseFileReader, isMetadataTable} +import org.apache.hudi.HoodieBaseRelation.createBaseFileReader import org.apache.hudi.common.model.{HoodieLogFile, HoodieRecord} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes -import org.apache.hudi.metadata.HoodieMetadataPayload import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile} -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{Row, SQLContext} import scala.collection.JavaConverters._ @@ -46,10 +43,6 @@ case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile], maxCompactionMemoryInBytes: Long, mergeType: String) -case class HoodieMergeOnReadTableState(hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit], - recordKeyField: String, - preCombineFieldOpt: Option[String]) - class MergeOnReadSnapshotRelation(sqlContext: SQLContext, optParams: Map[String, String], val userSchema: Option[StructType], @@ -57,38 +50,13 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, val metaClient: HoodieTableMetaClient) extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) { - private val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - private val jobConf = new JobConf(conf) - private val mergeType = optParams.getOrElse( DataSourceReadOptions.REALTIME_MERGE.key, DataSourceReadOptions.REALTIME_MERGE.defaultValue) private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf) - // If meta fields are enabled, always prefer key from the meta field as opposed to user-specified one - // NOTE: This is historical behavior which is preserved as is - private val recordKeyField = { - if (metaClient.getTableConfig.populateMetaFields()) HoodieRecord.RECORD_KEY_METADATA_FIELD - else metaClient.getTableConfig.getRecordKeyFieldProp - } - - private val preCombineFieldOpt = getPrecombineFieldProperty - - private lazy val mandatoryColumns = { - if (isMetadataTable(metaClient)) { - Seq(HoodieMetadataPayload.KEY_FIELD_NAME, HoodieMetadataPayload.SCHEMA_FIELD_NAME_TYPE) - } else { - Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq()) - } - } - - override def needConversion: Boolean = false - - private val specifiedQueryInstant = optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key) - .map(HoodieSqlCommonUtils.formatQueryInstant) - - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { + override def doBuildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[InternalRow] = { log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}") log.debug(s" buildScan filters = ${filters.mkString(",")}") @@ -137,12 +105,10 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, hadoopConf = new Configuration(conf) ) - val tableState = HoodieMergeOnReadTableState(fileIndex, recordKeyField, preCombineFieldOpt) + val tableState = HoodieTableState(recordKeyField, preCombineFieldOpt) - val rdd = new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, - requiredSchemaParquetReader, tableState, tableSchema, requiredSchema) - - rdd.asInstanceOf[RDD[Row]] + new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, + requiredSchemaParquetReader, tableState, tableSchema, requiredSchema, fileIndex) } def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] = { @@ -193,7 +159,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, val partitionColumns = hoodieFileIndex.partitionSchema.fieldNames.toSet val partitionFilters = filters.filter(f => f.references.forall(p => partitionColumns.contains(p))) val partitionFilterExpression = - HoodieSparkUtils.convertToCatalystExpressions(partitionFilters, tableStructSchema) + HoodieSparkUtils.convertToCatalystExpression(partitionFilters, tableStructSchema) val convertedPartitionFilterExpression = HoodieFileIndex.convertFilterForTimestampKeyGenerator(metaClient, partitionFilterExpression.toSeq) @@ -231,11 +197,6 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext, } } } - - private def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = { - val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col)) - requestedColumns ++ missing - } } object MergeOnReadSnapshotRelation { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 46201c4132078..d4c50b73fd402 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -146,15 +146,25 @@ class SparkHoodieTableFileIndex(spark: SparkSession, }).toMap } + /** + * Get all the cached partition paths pruned by the filter. + * + * @param predicates The filter condition + * @return The pruned partition paths + */ + def getPartitionPaths(predicates: Seq[Expression]): Seq[PartitionPath] = { + prunePartition(cachedAllInputFileSlices.keySet().asScala.toSeq, predicates) + } + /** * Prune the partition by the filter.This implementation is fork from * org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex#prunePartitions. * * @param partitionPaths All the partition paths. * @param predicates The filter condition. - * @return The Pruned partition paths. + * @return The pruned partition paths. */ - def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = { + protected def prunePartition(partitionPaths: Seq[PartitionPath], predicates: Seq[Expression]): Seq[PartitionPath] = { val partitionColumnNames = partitionSchema.fields.map(_.name).toSet val partitionPruningPredicates = predicates.filter { _.references.map(_.name).toSet.subsetOf(partitionColumnNames) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieUnsafeRDDUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieUnsafeRDDUtils.scala new file mode 100644 index 0000000000000..1ac8fa098119f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/HoodieUnsafeRDDUtils.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.apache.hudi.HoodieUnsafeRDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.util.MutablePair + +/** + * Suite of utilities helping in handling instances of [[HoodieUnsafeRDD]] + */ +object HoodieUnsafeRDDUtils { + + /** + * Canonical implementation of the [[RDD#collect]] for [[HoodieUnsafeRDD]], returning a properly + * copied [[Array]] of [[InternalRow]]s + */ + def collect(rdd: HoodieUnsafeRDD): Array[InternalRow] = { + rdd.mapPartitionsInternal { iter => + // NOTE: We're leveraging [[MutablePair]] here to avoid unnecessary allocations, since + // a) iteration is performed lazily and b) iteration is single-threaded (w/in partition) + val pair = new MutablePair[InternalRow, Null]() + iter.map(row => pair.update(row.copy(), null)) + } + .map(p => p._1) + .collect() + } + +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala new file mode 100644 index 0000000000000..d640c02261742 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation} +import org.apache.spark.sql.types.StructType + +object HoodieCatalystExpressionUtils { + + /** + * Resolve filter expression from string expr with given table schema, for example: + *
+   *   ts > 1000 and ts <= 1500
+   * 
+ * will be resolved as + *
+   *   And(GreaterThan(ts#590L > 1000), LessThanOrEqual(ts#590L <= 1500))
+   * 
+ * + * @param spark The spark session + * @param exprString String to be resolved + * @param tableSchema The table schema + * @return Resolved filter expression + */ + def resolveFilterExpr(spark: SparkSession, exprString: String, tableSchema: StructType): Expression = { + val expr = spark.sessionState.sqlParser.parseExpression(exprString) + resolveFilterExpr(spark, expr, tableSchema) + } + + def resolveFilterExpr(spark: SparkSession, expr: Expression, tableSchema: StructType): Expression = { + val schemaFields = tableSchema.fields + val resolvedExpr = spark.sessionState.analyzer.ResolveReferences( + Filter(expr, + LocalRelation(schemaFields.head, schemaFields.drop(1): _*)) + ) + .asInstanceOf[Filter].condition + + checkForUnresolvedRefs(resolvedExpr) + } + + private def checkForUnresolvedRefs(resolvedExpr: Expression): Expression = + resolvedExpr match { + case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute") + case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e)) + } + + /** + * Split the given predicates into two sequence predicates: + * - predicates that references partition columns only(and involves no sub-query); + * - other predicates. + * + * @param sparkSession The spark session + * @param predicates The predicates to be split + * @param partitionColumns The partition columns + * @return (partitionFilters, dataFilters) + */ + def splitPartitionAndDataPredicates(sparkSession: SparkSession, + predicates: Array[Expression], + partitionColumns: Array[String]): (Array[Expression], Array[Expression]) = { + // Validates that the provided names both resolve to the same entity + val resolvedNameEquals = sparkSession.sessionState.analyzer.resolver + + predicates.partition(expr => { + // Checks whether given expression only references partition columns(and involves no sub-query) + expr.references.forall(r => partitionColumns.exists(resolvedNameEquals(r.name, _))) && + !SubqueryExpression.hasSubquery(expr) + }) + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieSparkAvroSerializer.scala similarity index 89% rename from hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala rename to hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieSparkAvroSerializer.scala index 050efbd3d22c2..4a3a7c4526dee 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieAvroSerializer.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/avro/HoodieSparkAvroSerializer.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.avro import org.apache.avro.Schema import org.apache.spark.sql.types.DataType -class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) - extends HoodieAvroSerializerTrait { +class HoodieSparkAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) + extends HoodieAvroSerializer { val avroSerializer = new AvroSerializer(rootCatalystType, rootAvroType, nullable) diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java index ce80b5232cfe4..9491e43e21fe8 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDataSourceHelpers.java @@ -18,12 +18,16 @@ package org.apache.hudi; +import org.apache.hudi.avro.model.HoodieClusteringPlan; 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.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; 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.hadoop.fs.FileSystem; @@ -80,4 +84,17 @@ public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, Strin return metaClient.getCommitTimeline().filterCompletedInstants(); } } + + @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) + public static Option getClusteringPlan(FileSystem fs, String basePath, String instantTime) { + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()) + .setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieInstant hoodieInstant = HoodieTimeline.getReplaceCommitRequestedInstant(instantTime); + Option> clusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, hoodieInstant); + if (clusteringPlan.isPresent()) { + return Option.of(clusteringPlan.get().getValue()); + } else { + return Option.empty(); + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala index 28f8a92e94405..b140b6767e1e4 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.hudi.analysis import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.util.ReflectionUtils -import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} +import org.apache.hudi.{DataSourceReadOptions, HoodieSparkUtils, SparkAdapterSupport} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.catalog.{CatalogUtils, HoodieCatalogTable} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, GenericInternalRow, Literal, NamedExpression} import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{getTableIdentifier, removeMetaFields} import org.apache.spark.sql.hudi.HoodieSqlUtils._ import org.apache.spark.sql.hudi.command._ @@ -113,6 +114,7 @@ case class HoodieAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan] case _ => l } + // Convert to CreateHoodieTableAsSelectCommand case CreateTable(table, mode, Some(query)) if query.resolved && sparkAdapter.isHoodieTable(table) => @@ -396,6 +398,37 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi l } + case l if sparkAdapter.isRelationTimeTravel(l) => + val (plan: UnresolvedRelation, timestamp, version) = + sparkAdapter.getRelationTimeTravel(l).get + + if (timestamp.isEmpty && version.nonEmpty) { + throw new AnalysisException( + "version expression is not supported for time travel") + } + + val tableIdentifier = sparkAdapter.toTableIdentifier(plan) + if (sparkAdapter.isHoodieTable(tableIdentifier, sparkSession)) { + val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier) + val table = hoodieCatalogTable.table + val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + val instantOption = Map( + DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key -> timestamp.get.toString()) + val dataSource = + DataSource( + sparkSession, + userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema), + partitionColumns = table.partitionColumnNames, + bucketSpec = table.bucketSpec, + className = table.provider.get, + options = table.storage.properties ++ pathOption ++ instantOption, + catalogTable = Some(table)) + + LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table) + } else { + l + } + case p => p } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala index 2f5c4d004f58f..1135981a92acd 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala @@ -17,17 +17,15 @@ package org.apache.spark.sql.hudi.command +import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieTableType} +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{HoodieTimer, Option => HOption} import org.apache.hudi.exception.HoodieException -import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils} -import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE} -import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.types.StringType import org.apache.spark.sql.{Row, SparkSession} @@ -44,19 +42,7 @@ case class CompactionHoodiePathCommand(path: String, assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ, s"Must compaction on a Merge On Read table.") - val schemaUtil = new TableSchemaResolver(metaClient) - val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString - - val parameters = HoodieWriterUtils.parametersWithWriteDefaults( - HoodieSqlCommonUtils.withSparkConf(sparkSession, Map.empty)( - Map( - DataSourceWriteOptions.TABLE_TYPE.key() -> HoodieTableType.MERGE_ON_READ.name() - ) - ) - ) - val jsc = new JavaSparkContext(sparkSession.sparkContext) - val client = DataSourceUtils.createHoodieClient(jsc, schemaStr, path, - metaClient.getTableConfig.getTableName, parameters) + val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, path, Map.empty) operation match { case SCHEDULE => diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala index e64df997da2ff..876bb503ee196 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/BaseProcedure.scala @@ -21,12 +21,18 @@ import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.model.HoodieRecordPayload import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} +import org.apache.hudi.exception.HoodieClusteringException import org.apache.hudi.index.HoodieIndex.IndexType import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.types._ +import java.nio.charset.Charset +import java.sql.{Date, Timestamp} + abstract class BaseProcedure extends Procedure { val INVALID_ARG_INDEX: Int = -1 @@ -68,14 +74,22 @@ abstract class BaseProcedure extends Procedure { args.map.getOrDefault(key, INVALID_ARG_INDEX) } - protected def getArgValueOrDefault(args: ProcedureArgs, parameter: ProcedureParameter): Any = { + protected def getArgValueOrDefault(args: ProcedureArgs, parameter: ProcedureParameter): Option[Any] = { var argsIndex: Int = INVALID_ARG_INDEX if (args.isNamedArgs) { argsIndex = getArgsIndex(parameter.name, args) } else { argsIndex = getArgsIndex(parameter.index.toString, args) } - if (argsIndex.equals(INVALID_ARG_INDEX)) parameter.default else getInternalRowValue(args.internalRow, argsIndex, parameter.dataType) + + if (argsIndex.equals(INVALID_ARG_INDEX)) { + parameter.default match { + case option: Option[Any] => option + case _ => Option.apply(parameter.default) + } + } else { + Option.apply(getInternalRowValue(args.internalRow, argsIndex, parameter.dataType)) + } } protected def getInternalRowValue(row: InternalRow, index: Int, dataType: DataType): Any = { @@ -96,4 +110,40 @@ abstract class BaseProcedure extends Procedure { throw new UnsupportedOperationException(s"type: ${dataType.typeName} not supported") } } + + protected def getBasePath(tableName: Option[Any], tablePath: Option[Any] = Option.empty): String = { + tableName.map( + t => HoodieCatalogTable(sparkSession, new TableIdentifier(t.asInstanceOf[String])).tableLocation) + .getOrElse( + tablePath.map(p => p.asInstanceOf[String]).getOrElse( + throw new HoodieClusteringException("Table name or table path must be given one")) + ) + } + + protected def convertCatalystType(value: String, dataType: DataType): Any = { + try { + val valueWithType = dataType match { + case StringType => value + case BinaryType => value.getBytes(Charset.forName("utf-8")) + case BooleanType => value.toBoolean + case DoubleType => value.toDouble + case d: DecimalType => Decimal.apply(BigDecimal(value), d.precision, d.scale) + case FloatType => value.toFloat + case ByteType => value.toByte + case IntegerType => value.toInt + case LongType => value.toLong + case ShortType => value.toShort + case DateType => DateTimeUtils.fromJavaDate(Date.valueOf(value)) + case TimestampType => DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(value)) + case _ => throw new HoodieClusteringException("Data type not support:" + dataType) + } + + valueWithType + } catch { + case e: HoodieClusteringException => + throw e + case _ => + throw new HoodieClusteringException("Data type not match, value:" + value + ", dataType:" + dataType) + } + } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointsProcedure.scala new file mode 100644 index 0000000000000..ed4905ed047a7 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointsProcedure.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.exception.{HoodieException, HoodieSavepointException} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util.function.Supplier + +class CreateSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "commit_Time", DataTypes.StringType, None), + ProcedureParameter.optional(2, "user", DataTypes.StringType, ""), + ProcedureParameter.optional(3, "comments", DataTypes.StringType, "") + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("create_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val commitTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + val user = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String] + val comments = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[String] + + val basePath: String = getBasePath(tableName) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val activeTimeline: HoodieActiveTimeline = metaClient.getActiveTimeline + if (!activeTimeline.getCommitsTimeline.filterCompletedInstants.containsInstant(commitTime)) { + throw new HoodieException("Commit " + commitTime + " not found in Commits " + activeTimeline) + } + + val client = createHoodieClient(jsc, basePath) + var result = false + + try { + client.savepoint(commitTime, user, comments) + logInfo(s"The commit $commitTime has been savepointed.") + result = true + } catch { + case _: HoodieSavepointException => + logWarning(s"Failed: Could not create savepoint $commitTime.") + } + + Seq(Row(result)) + } + + override def build: Procedure = new CreateSavepointsProcedure() +} + +object CreateSavepointsProcedure { + val NAME: String = "create_savepoints" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get(): CreateSavepointsProcedure = new CreateSavepointsProcedure() + } +} + + + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointsProcedure.scala new file mode 100644 index 0000000000000..11416ac22c56f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointsProcedure.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.exception.{HoodieException, HoodieSavepointException} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util.function.Supplier + +class DeleteSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "instant_time", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("delete_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + + val basePath: String = getBasePath(tableName) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants + if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete") + val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime) + + if (!completedInstants.containsInstant(savePoint)) { + throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants) + } + + val client = createHoodieClient(jsc, basePath) + var result = false + + try { + client.deleteSavepoint(instantTime) + logInfo(s"The commit $instantTime has been deleted savepoint.") + result = true + } catch { + case _: HoodieSavepointException => + logWarning(s"Failed: Could not delete savepoint $instantTime.") + } + + Seq(Row(result)) + } + + override def build: Procedure = new DeleteSavepointsProcedure() +} + +object DeleteSavepointsProcedure { + val NAME: String = "delete_savepoints" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get(): DeleteSavepointsProcedure = new DeleteSavepointsProcedure() + } +} + + + + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala index 7b919fcef08b5..9c05773531322 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/HoodieProcedures.scala @@ -33,9 +33,15 @@ object HoodieProcedures { private def initProcedureBuilders: util.Map[String, Supplier[ProcedureBuilder]] = { val mapBuilder: ImmutableMap.Builder[String, Supplier[ProcedureBuilder]] = ImmutableMap.builder() + mapBuilder.put(CreateSavepointsProcedure.NAME, CreateSavepointsProcedure.builder) + mapBuilder.put(DeleteSavepointsProcedure.NAME, DeleteSavepointsProcedure.builder) + mapBuilder.put(RollbackSavepointsProcedure.NAME, RollbackSavepointsProcedure.builder) + mapBuilder.put(RollbackToInstantTimeProcedure.NAME, RollbackToInstantTimeProcedure.builder) + mapBuilder.put(RunClusteringProcedure.NAME, RunClusteringProcedure.builder) + mapBuilder.put(ShowClusteringProcedure.NAME, ShowClusteringProcedure.builder) mapBuilder.put(ShowCommitsProcedure.NAME, ShowCommitsProcedure.builder) mapBuilder.put(ShowCommitsMetadataProcedure.NAME, ShowCommitsMetadataProcedure.builder) - mapBuilder.put(RollbackToInstantTimeProcedure.NAME, RollbackToInstantTimeProcedure.builder) + mapBuilder.put(ShowSavepointsProcedure.NAME, ShowSavepointsProcedure.builder) mapBuilder.build } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackSavepointsProcedure.scala new file mode 100644 index 0000000000000..a11e614176f57 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackSavepointsProcedure.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.exception.{HoodieException, HoodieSavepointException} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util.function.Supplier + +class RollbackSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None), + ProcedureParameter.required(1, "instant_time", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("rollback_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] + + val basePath: String = getBasePath(tableName) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants + if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete") + val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime) + + if (!completedInstants.containsInstant(savePoint)) { + throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants) + } + + val client = createHoodieClient(jsc, basePath) + var result = false + + try { + client.restoreToSavepoint(instantTime) + logInfo("The commit $instantTime rolled back.") + result = true + } catch { + case _: HoodieSavepointException => + logWarning(s"The commit $instantTime failed to roll back.") + } + + Seq(Row(result)) + } + + override def build: Procedure = new RollbackSavepointsProcedure() +} + +object RollbackSavepointsProcedure { + val NAME: String = "rollback_savepoints" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get(): RollbackSavepointsProcedure = new RollbackSavepointsProcedure() + } +} + + + + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToInstantTimeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToInstantTimeProcedure.scala index 5414e8db6b37d..f17efe2449607 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToInstantTimeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToInstantTimeProcedure.scala @@ -45,8 +45,8 @@ class RollbackToInstantTimeProcedure extends BaseProcedure with ProcedureBuilder override def call(args: ProcedureArgs): Seq[Row] = { super.checkArgs(PARAMETERS, args) - val table = getArgValueOrDefault(args, PARAMETERS(0)).asInstanceOf[String] - val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).asInstanceOf[String] + val table = getArgValueOrDefault(args, PARAMETERS(0)).get.asInstanceOf[String] + val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String] val hoodieCatalogTable = HoodieCatalogTable(sparkSession, new TableIdentifier(table)) val basePath = hoodieCatalogTable.tableLocation diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala new file mode 100644 index 0000000000000..442ee04415c84 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala @@ -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.spark.sql.hudi.command.procedures + +import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL} +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} +import org.apache.hudi.common.util.ValidationUtils.checkArgument +import org.apache.hudi.common.util.{ClusteringUtils, Option => HOption} +import org.apache.hudi.config.HoodieClusteringConfig +import org.apache.hudi.exception.HoodieClusteringException +import org.apache.hudi.{AvroConversionUtils, HoodieCLIUtils, HoodieFileIndex} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row} +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.execution.datasources.FileStatusCache +import org.apache.spark.sql.types._ + +import java.util.function.Supplier +import scala.collection.JavaConverters._ + +class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with PredicateHelper with Logging { + /** + * OPTIMIZE table_name|table_path [WHERE predicate] + * [ORDER BY (col_name1 [, ...] ) ] + */ + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.optional(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "path", DataTypes.StringType, None), + ProcedureParameter.optional(2, "predicate", DataTypes.StringType, None), + ProcedureParameter.optional(3, "order", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("partition", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty) + )) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) + val predicate = getArgValueOrDefault(args, PARAMETERS(2)) + val orderColumns = getArgValueOrDefault(args, PARAMETERS(3)) + + val basePath: String = getBasePath(tableName, tablePath) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + var conf: Map[String, String] = Map.empty + predicate match { + case Some(p) => + val prunedPartitions = prunePartition(metaClient, p.asInstanceOf[String]) + conf = conf ++ Map( + HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key() -> "SELECTED_PARTITIONS", + HoodieClusteringConfig.PARTITION_SELECTED.key() -> prunedPartitions + ) + logInfo(s"Partition predicates: ${p}, partition selected: ${prunedPartitions}") + case _ => + logInfo("No partition predicates") + } + + // Construct sort column info + orderColumns match { + case Some(o) => + validateOrderColumns(o.asInstanceOf[String], metaClient) + conf = conf ++ Map( + HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key() -> o.asInstanceOf[String] + ) + logInfo(s"Order columns: ${o}") + case _ => + logInfo("No order columns") + } + + // Get all pending clustering instants + var pendingClustering = ClusteringUtils.getAllPendingClusteringPlans(metaClient) + .iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f) + logInfo(s"Pending clustering instants: ${pendingClustering.mkString(",")}") + + val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, basePath, conf) + val instantTime = HoodieActiveTimeline.createNewInstantTime + if (client.scheduleClusteringAtInstant(instantTime, HOption.empty())) { + pendingClustering ++= Seq(instantTime) + } + logInfo(s"Clustering instants to run: ${pendingClustering.mkString(",")}.") + + val startTs = System.currentTimeMillis() + pendingClustering.foreach(client.cluster(_, true)) + logInfo(s"Finish clustering all the instants: ${pendingClustering.mkString(",")}," + + s" time cost: ${System.currentTimeMillis() - startTs}ms.") + Seq.empty[Row] + } + + override def build: Procedure = new RunClusteringProcedure() + + def prunePartition(metaClient: HoodieTableMetaClient, predicate: String): String = { + val options = Map(QUERY_TYPE.key() -> QUERY_TYPE_SNAPSHOT_OPT_VAL, "path" -> metaClient.getBasePath) + val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, None, options, + FileStatusCache.getOrCreate(sparkSession)) + + // Resolve partition predicates + val schemaResolver = new TableSchemaResolver(metaClient) + val tableSchema = AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema) + val condition = HoodieCatalystExpressionUtils.resolveFilterExpr(sparkSession, predicate, tableSchema) + val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]()) + val (partitionPredicates, dataPredicates) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates( + sparkSession, splitConjunctivePredicates(condition).toArray, partitionColumns) + checkArgument(dataPredicates.isEmpty, "Only partition predicates are allowed") + + // Get all partitions and prune partition by predicates + val prunedPartitions = hoodieFileIndex.getPartitionPaths(partitionPredicates) + prunedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",") + } + + private def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = { + if (orderColumns == null) { + throw new HoodieClusteringException("Order columns is null") + } + + val tableSchemaResolver = new TableSchemaResolver(metaClient) + val fields = tableSchemaResolver.getTableAvroSchema(false) + .getFields.asScala.map(_.name().toLowerCase) + orderColumns.split(",").foreach(col => { + if (!fields.contains(col.toLowerCase)) { + throw new HoodieClusteringException("Order column not exist:" + col) + } + }) + } + +} + +object RunClusteringProcedure { + val NAME = "run_clustering" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new RunClusteringProcedure + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala new file mode 100644 index 0000000000000..a9d808217c0a9 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.SparkAdapterSupport +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.util.ClusteringUtils +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ + +import java.util.function.Supplier +import scala.collection.JavaConverters._ + +class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with SparkAdapterSupport with Logging { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.optional(0, "table", DataTypes.StringType, None), + ProcedureParameter.optional(1, "path", DataTypes.StringType, None), + ProcedureParameter.optional(2, "limit", DataTypes.IntegerType, 20) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("timestamp", DataTypes.StringType, nullable = true, Metadata.empty), + StructField("groups", DataTypes.IntegerType, nullable = true, Metadata.empty) + )) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + val tablePath = getArgValueOrDefault(args, PARAMETERS(1)) + val limit = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[Int] + + val basePath: String = getBasePath(tableName, tablePath) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + ClusteringUtils.getAllPendingClusteringPlans(metaClient).iterator().asScala.map { p => + Row(p.getLeft.getTimestamp, p.getRight.getInputGroups.size()) + }.toSeq.take(limit) + } + + override def build: Procedure = new ShowClusteringProcedure() +} + +object ShowClusteringProcedure { + val NAME = "show_clustering" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get() = new ShowClusteringProcedure + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala index da089baba9cb6..920e3a2c16cf3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala @@ -75,8 +75,8 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure override def call(args: ProcedureArgs): Seq[Row] = { super.checkArgs(PARAMETERS, args) - val table = getArgValueOrDefault(args, PARAMETERS(0)).asInstanceOf[String] - val limit = getArgValueOrDefault(args, PARAMETERS(1)).asInstanceOf[Int] + val table = getArgValueOrDefault(args, PARAMETERS(0)).get.asInstanceOf[String] + val limit = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[Int] val hoodieCatalogTable = HoodieCatalogTable(sparkSession, new TableIdentifier(table)) val basePath = hoodieCatalogTable.tableLocation diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala new file mode 100644 index 0000000000000..e866e21555baf --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi.command.procedures + +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline} +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} + +import java.util +import java.util.function.Supplier +import java.util.stream.Collectors + +class ShowSavepointsProcedure extends BaseProcedure with ProcedureBuilder { + private val PARAMETERS = Array[ProcedureParameter]( + ProcedureParameter.required(0, "table", DataTypes.StringType, None) + ) + + private val OUTPUT_TYPE = new StructType(Array[StructField]( + StructField("savepoint_time", DataTypes.StringType, nullable = true, Metadata.empty)) + ) + + def parameters: Array[ProcedureParameter] = PARAMETERS + + def outputType: StructType = OUTPUT_TYPE + + override def call(args: ProcedureArgs): Seq[Row] = { + super.checkArgs(PARAMETERS, args) + + val tableName = getArgValueOrDefault(args, PARAMETERS(0)) + + val basePath: String = getBasePath(tableName) + val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build + + val activeTimeline: HoodieActiveTimeline = metaClient.getActiveTimeline + val timeline: HoodieTimeline = activeTimeline.getSavePointTimeline.filterCompletedInstants + val commits: util.List[HoodieInstant] = timeline.getReverseOrderedInstants.collect(Collectors.toList[HoodieInstant]) + + if (commits.isEmpty) Seq.empty[Row] else { + commits.toArray.map(instant => instant.asInstanceOf[HoodieInstant].getTimestamp).map(p => Row(p)).toSeq + } + } + + override def build: Procedure = new ShowSavepointsProcedure() +} + +object ShowSavepointsProcedure { + val NAME: String = "show_savepoints" + + def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] { + override def get(): ShowSavepointsProcedure = new ShowSavepointsProcedure() + } +} + diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala index 3146740b1f3f5..771798dd225e9 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieSqlCommonAstBuilder.scala @@ -61,20 +61,20 @@ class HoodieSqlCommonAstBuilder(session: SparkSession, delegate: ParserInterface CompactionTable(table, operation, timestamp) } - override def visitCompactionOnPath (ctx: CompactionOnPathContext): LogicalPlan = withOrigin(ctx) { + override def visitCompactionOnPath(ctx: CompactionOnPathContext): LogicalPlan = withOrigin(ctx) { val path = string(ctx.path) val operation = CompactionOperation.withName(ctx.operation.getText.toUpperCase) val timestamp = if (ctx.instantTimestamp != null) Some(ctx.instantTimestamp.getText.toLong) else None CompactionPath(path, operation, timestamp) } - override def visitShowCompactionOnTable (ctx: ShowCompactionOnTableContext): LogicalPlan = withOrigin(ctx) { + override def visitShowCompactionOnTable(ctx: ShowCompactionOnTableContext): LogicalPlan = withOrigin(ctx) { val table = ctx.tableIdentifier().accept(this).asInstanceOf[LogicalPlan] - if (ctx.limit != null) { - CompactionShowOnTable(table, ctx.limit.getText.toInt) - } else { - CompactionShowOnTable(table) - } + if (ctx.limit != null) { + CompactionShowOnTable(table, ctx.limit.getText.toInt) + } else { + CompactionShowOnTable(table) + } } override def visitShowCompactionOnPath(ctx: ShowCompactionOnPathContext): LogicalPlan = withOrigin(ctx) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 5baaffab0cf7c..4af3943966aa3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -364,6 +364,7 @@ public void stream(Dataset streamingInput, String operationType, String che .option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1") .option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "true") .option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true") + .option(HoodieCompactionConfig.PRESERVE_COMMIT_METADATA.key(), "false") .option(HoodieWriteConfig.TBL_NAME.key(), tableName).option("checkpointLocation", checkpointLocation) .outputMode(OutputMode.Append()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index bf3520f0956d7..af5bbe7717959 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; @@ -222,6 +224,30 @@ public void testCreateRDDCustomColumnsSortPartitionerWithValidPartitioner() thro assertThat(partitioner.isPresent(), is(true)); } + @Test + public void testCreateHoodieConfigWithAsyncClustering() { + ArrayList> asyncClusteringKeyValues = new ArrayList<>(4); + asyncClusteringKeyValues.add(new ImmutablePair(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), true)); + asyncClusteringKeyValues.add(new ImmutablePair(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE.key(), true)); + asyncClusteringKeyValues.add(new ImmutablePair("hoodie.datasource.clustering.async.enable", true)); + asyncClusteringKeyValues.add(new ImmutablePair("hoodie.clustering.async.enabled", true)); + + asyncClusteringKeyValues.stream().forEach(pair -> { + HashMap params = new HashMap<>(3); + params.put(DataSourceWriteOptions.TABLE_TYPE().key(), DataSourceWriteOptions.TABLE_TYPE().defaultValue()); + params.put(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key(), + DataSourceWriteOptions.PAYLOAD_CLASS_NAME().defaultValue()); + params.put(pair.left, pair.right.toString()); + HoodieWriteConfig hoodieConfig = DataSourceUtils + .createHoodieConfig(avroSchemaString, config.getBasePath(), "test", params); + assertEquals(pair.right, hoodieConfig.isAsyncClusteringEnabled()); + + TypedProperties prop = new TypedProperties(); + prop.putAll(params); + assertEquals(pair.right, HoodieClusteringConfig.from(prop).isAsyncClusteringEnabled()); + }); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void testBuildHiveSyncConfig(boolean useSyncMode) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/SparkRDDWriteClientOverride.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/SparkRDDWriteClientOverride.java new file mode 100644 index 0000000000000..20982b5cda688 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/SparkRDDWriteClientOverride.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.functional; + +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.config.HoodieWriteConfig; + +// Sole purpose of this class is to provide access to otherwise API inaccessible from the tests. +// While it's certainly not a great pattern, it would require substantial test restructuring to +// eliminate such access to an internal API, so this is considered acceptable given it's very limited +// scope (w/in the current package) +class SparkRDDWriteClientOverride extends org.apache.hudi.client.SparkRDDWriteClient { + + public SparkRDDWriteClientOverride(HoodieEngineContext context, HoodieWriteConfig clientConfig) { + super(context, clientConfig); + } + + @Override + public void rollbackFailedBootstrap() { + super.rollbackFailedBootstrap(); + } +} + diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index d2257f58d0e80..6b54765a0b08b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -20,7 +20,6 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; @@ -253,10 +252,11 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec .withBootstrapParallelism(3) .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) .build(); - SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + + SparkRDDWriteClientOverride client = new SparkRDDWriteClientOverride(context, config); client.bootstrap(Option.empty()); checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true); // Rollback Bootstrap HoodieActiveTimeline.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, @@ -272,7 +272,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec assertFalse(index.useIndex()); // Run bootstrap again - client = new SparkRDDWriteClient(context, config); + client = new SparkRDDWriteClientOverride(context, config); client.bootstrap(Option.empty()); metaClient.reloadActiveTimeline(); @@ -284,7 +284,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec } checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true); // Upsert case long updateTimestamp = Instant.now().toEpochMilli(); @@ -296,7 +296,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec String newInstantTs = client.startCommit(); client.upsert(updateBatch, newInstantTs); checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, - updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); + updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit, true); if (deltaCommit) { Option compactionInstant = client.scheduleCompaction(Option.empty()); @@ -304,7 +304,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec client.compact(compactionInstant.get()); checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, - Arrays.asList(compactionInstant.get())); + Arrays.asList(compactionInstant.get()), !config.isPreserveHoodieCommitMetadataForCompaction()); } } @@ -334,14 +334,14 @@ public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { } private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, - int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception { + int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, boolean validateRecordsForCommitTime) throws Exception { checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, - expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); + expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant), validateRecordsForCommitTime); } private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, - List instantsWithValidRecords) throws Exception { + List instantsWithValidRecords, boolean validateRecordsForCommitTime) throws Exception { metaClient.reloadActiveTimeline(); assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); assertEquals(instant, metaClient.getActiveTimeline() @@ -361,8 +361,10 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta if (!isDeltaCommit) { String predicate = String.join(", ", instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); - assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " - + "(" + predicate + ")").count()); + if (validateRecordsForCommitTime) { + assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " + + "(" + predicate + ")").count()); + } Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " + "in (select _hoodie_record_key from bootstrapped)"); assertEquals(0, missingOriginal.count()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java index 9146cdc4e81f7..330b6015bc625 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java @@ -20,7 +20,6 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; @@ -245,10 +244,11 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec .withBootstrapParallelism(3) .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) .build(); - SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + + SparkRDDWriteClientOverride client = new SparkRDDWriteClientOverride(context, config); client.bootstrap(Option.empty()); checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true); // Rollback Bootstrap if (deltaCommit) { @@ -266,7 +266,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec assertFalse(index.useIndex()); // Run bootstrap again - client = new SparkRDDWriteClient(context, config); + client = new SparkRDDWriteClientOverride(context, config); client.bootstrap(Option.empty()); metaClient.reloadActiveTimeline(); @@ -278,7 +278,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec } checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true); // Upsert case long updateTimestamp = Instant.now().toEpochMilli(); @@ -290,7 +290,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec String newInstantTs = client.startCommit(); client.upsert(updateBatch, newInstantTs); checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, - updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); + updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit, true); if (deltaCommit) { Option compactionInstant = client.scheduleCompaction(Option.empty()); @@ -298,7 +298,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec client.compact(compactionInstant.get()); checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, - Arrays.asList(compactionInstant.get())); + Arrays.asList(compactionInstant.get()), !config.isPreserveHoodieCommitMetadataForCompaction()); } } @@ -328,14 +328,14 @@ public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { } private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, - int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception { + int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, boolean validateRecordsForCommitTime) throws Exception { checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, - expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); + expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant), validateRecordsForCommitTime); } private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, - List instantsWithValidRecords) throws Exception { + List instantsWithValidRecords, boolean validateCommitRecords) throws Exception { metaClient.reloadActiveTimeline(); assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); assertEquals(instant, metaClient.getActiveTimeline() @@ -355,8 +355,10 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta if (!isDeltaCommit) { String predicate = String.join(", ", instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); - assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " - + "(" + predicate + ")").count()); + if (validateCommitRecords) { + assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " + + "(" + predicate + ")").count()); + } Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " + "in (select _hoodie_record_key from bootstrapped)"); assertEquals(0, missingOriginal.count()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala index 50137c9a580a3..bacd44753df35 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestAvroConversionUtils.scala @@ -161,4 +161,220 @@ class TestAvroConversionUtils extends FunSuite with Matchers { assert(avroSchema.equals(expectedAvroSchema)) } + + test("test convertStructTypeToAvroSchema with Nested StructField comment") { + val mapType = DataTypes.createMapType(StringType, new StructType().add("mapKey", "string", false, "mapKeyComment").add("mapVal", "integer", true)) + val arrayType = ArrayType(new StructType().add("arrayKey", "string", false).add("arrayVal", "integer", true, "arrayValComment")) + val innerStruct = new StructType().add("innerKey","string",false, "innerKeyComment").add("value", "long", true, "valueComment") + + val struct = new StructType().add("key", "string", false).add("version", "string", true, "versionComment") + .add("data1",innerStruct,false).add("data2",innerStruct,true) + .add("nullableMap", mapType, true).add("map",mapType,false) + .add("nullableArray", arrayType, true).add("array",arrayType,false) + + val avroSchema = AvroConversionUtils.convertStructTypeToAvroSchema(struct, "SchemaName", "SchemaNS") + + val expectedSchemaStr = s""" + { + "type": "record", + "name": "SchemaName", + "namespace": "SchemaNS", + "fields": [ + { + "name": "key", + "type": "string" + }, + { + "name": "version", + "type": [ + "null", + "string" + ], + "doc": "versionComment", + "default": null + }, + { + "name": "data1", + "type": { + "type": "record", + "name": "data1", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "innerKey", + "type": "string", + "doc": "innerKeyComment" + }, + { + "name": "value", + "type": [ + "null", + "long" + ], + "doc": "valueComment", + "default": null + } + ] + } + }, + { + "name": "data2", + "type": [ + "null", + { + "type": "record", + "name": "data2", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "innerKey", + "type": "string", + "doc": "innerKeyComment" + }, + { + "name": "value", + "type": [ + "null", + "long" + ], + "doc": "valueComment", + "default": null + } + ] + } + ], + "default": null + }, + { + "name": "nullableMap", + "type": [ + "null", + { + "type": "map", + "values": [ + { + "type": "record", + "name": "nullableMap", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "mapKey", + "type": "string", + "doc": "mapKeyComment" + }, + { + "name": "mapVal", + "type": [ + "null", + "int" + ], + "default": null + } + ] + }, + "null" + ] + } + ], + "default": null + }, + { + "name": "map", + "type": { + "type": "map", + "values": [ + { + "type": "record", + "name": "map", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "mapKey", + "type": "string", + "doc": "mapKeyComment" + }, + { + "name": "mapVal", + "type": [ + "null", + "int" + ], + "default": null + } + ] + }, + "null" + ] + } + }, + { + "name": "nullableArray", + "type": [ + "null", + { + "type": "array", + "items": [ + { + "type": "record", + "name": "nullableArray", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "arrayKey", + "type": "string" + }, + { + "name": "arrayVal", + "type": [ + "null", + "int" + ], + "doc": "arrayValComment", + "default": null + } + ] + }, + "null" + ] + } + ], + "default": null + }, + { + "name": "array", + "type": { + "type": "array", + "items": [ + { + "type": "record", + "name": "array", + "namespace": "SchemaNS.SchemaName", + "fields": [ + { + "name": "arrayKey", + "type": "string" + }, + { + "name": "arrayVal", + "type": [ + "null", + "int" + ], + "doc": "arrayValComment", + "default": null + } + ] + }, + "null" + ] + } + } + ] + }} + """ + + val expectedAvroSchema = new Schema.Parser().parse(expectedSchemaStr) + + assert(avroSchema.equals(expectedAvroSchema)) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala index 9b1b88d34ce18..8aa47ffc2f880 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestConvertFilterToCatalystExpression.scala @@ -17,11 +17,9 @@ package org.apache.hudi -import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpressions import org.apache.hudi.HoodieSparkUtils.convertToCatalystExpression - -import org.apache.spark.sql.sources.{And, EqualNullSafe, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Not, Or, StringContains, StringEndsWith, StringStartsWith} -import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Test @@ -93,7 +91,7 @@ class TestConvertFilterToCatalystExpression { } else { expectExpression } - val exp = convertToCatalystExpressions(filters, tableSchema) + val exp = convertToCatalystExpression(filters, tableSchema) if (removeQuotesIfNeed == null) { assertEquals(exp.isEmpty, true) } else { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala index 9e3572b561616..ac866ba3e4d70 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSkippingUtils.scala @@ -19,13 +19,11 @@ package org.apache.hudi import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper import org.apache.hudi.testutils.HoodieClientTestBase -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Expression, Not} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation} import org.apache.spark.sql.functions.col import org.apache.spark.sql.hudi.DataSkippingUtils import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType, VarcharType} -import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.{Column, HoodieCatalystExpressionUtils, SparkSession} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.params.ParameterizedTest @@ -75,8 +73,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase { @ParameterizedTest @MethodSource(Array("testBaseLookupFilterExpressionsSource", "testAdvancedLookupFilterExpressionsSource")) def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = { - val resolvedExpr: Expression = resolveFilterExpr(sourceExpr, sourceTableSchema) - + val resolvedExpr: Expression = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema) val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) val spark2 = spark @@ -96,7 +93,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase { @ParameterizedTest @MethodSource(Array("testStringsLookupFilterExpressionsSource")) def testStringsLookupFilterExpressions(sourceExpr: Expression, input: Seq[IndexRow], output: Seq[String]): Unit = { - val resolvedExpr = resolveFilterExpr(sourceExpr, sourceTableSchema) + val resolvedExpr = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema) val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema) val spark2 = spark @@ -112,27 +109,6 @@ class TestDataSkippingUtils extends HoodieClientTestBase { assertEquals(output, rows) } - - private def resolveFilterExpr(exprString: String, tableSchema: StructType): Expression = { - val expr = spark.sessionState.sqlParser.parseExpression(exprString) - resolveFilterExpr(expr, tableSchema) - } - - private def resolveFilterExpr(expr: Expression, tableSchema: StructType): Expression = { - val schemaFields = tableSchema.fields - val resolvedExpr = spark.sessionState.analyzer.ResolveReferences( - Filter(expr, LocalRelation(schemaFields.head, schemaFields.drop(1): _*)) - ) - .asInstanceOf[Filter].condition - - checkForUnresolvedRefs(resolvedExpr) - } - - def checkForUnresolvedRefs(resolvedExpr: Expression): Expression = - resolvedExpr match { - case UnresolvedAttribute(_) => throw new IllegalStateException("unresolved attribute") - case _ => resolvedExpr.mapChildren(e => checkForUnresolvedRefs(e)) - } } object TestDataSkippingUtils { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index ed6ef87b8e14f..5c20939cfb532 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -272,8 +272,9 @@ class TestMORDataSource extends HoodieClientTestBase { .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit5Time) .option(DataSourceReadOptions.END_INSTANTTIME.key, commit6Time) .load(basePath) - // compaction updated 150 rows + inserted 2 new row - assertEquals(152, hudiIncDF6.count()) + // even though compaction updated 150 rows, since preserve commit metadata is true, they won't be part of incremental query. + // inserted 2 new row + assertEquals(2, hudiIncDF6.count()) } @Test diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala index 315a14c9de03d..18b639f2f9bd2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala @@ -26,6 +26,7 @@ import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.testutils.SparkClientFunctionalTestHarness import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers} +import org.apache.log4j.LogManager import org.apache.spark.sql._ import org.apache.spark.sql.functions.{col, lit} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} @@ -39,6 +40,8 @@ import scala.collection.JavaConversions._ @Tag("functional") class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness { + private val log = LogManager.getLogger(classOf[TestMORDataSourceStorage]) + val commonOpts = Map( "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4", diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala new file mode 100644 index 0000000000000..a963081749455 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -0,0 +1,355 @@ +/* + * 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.functional + +import org.apache.avro.Schema +import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, DefaultSource, HoodieBaseRelation, HoodieSparkUtils, HoodieUnsafeRDD} +import org.apache.hudi.common.config.HoodieMetadataConfig +import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.testutils.{HadoopMapRedUtils, HoodieTestDataGenerator} +import org.apache.hudi.config.{HoodieStorageConfig, HoodieWriteConfig} +import org.apache.hudi.keygen.NonpartitionedKeyGenerator +import org.apache.hudi.testutils.SparkClientFunctionalTestHarness +import org.apache.parquet.hadoop.util.counters.BenchmarkCounter +import org.apache.spark.HoodieUnsafeRDDUtils +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{Dataset, Row, SaveMode} +import org.apache.spark.sql.catalyst.InternalRow +import org.junit.jupiter.api.Assertions.{assertEquals, fail} +import org.junit.jupiter.api.{Tag, Test} + +import scala.:+ +import scala.collection.JavaConverters._ + +@Tag("functional") +class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with Logging { + + val defaultWriteOpts = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + "hoodie.bulkinsert.shuffle.parallelism" -> "2", + "hoodie.delete.shuffle.parallelism" -> "1", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key", + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp", + HoodieWriteConfig.TBL_NAME.key -> "hoodie_test", + HoodieMetadataConfig.ENABLE.key -> "true", + // NOTE: It's critical that we use non-partitioned table, since the way we track amount of bytes read + // is not robust, and works most reliably only when we read just a single file. As such, making table + // non-partitioned makes it much more likely just a single file will be written + DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getName + ) + + @Test + def testBaseFileOnlyViewRelation(): Unit = { + val tablePath = s"$basePath/cow" + val targetRecordsCount = 100 + val (_, schema) = bootstrapTable(tablePath, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, targetRecordsCount, + defaultWriteOpts, populateMetaFields = true) + val tableState = TableState(tablePath, schema, targetRecordsCount, 0.0) + + // Stats for the reads fetching only _projected_ columns (note how amount of bytes read + // increases along w/ the # of columns) + val projectedColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 2452), + ("rider,driver", 2552), + ("rider,driver,tip_history", 3517)) + else if (HoodieSparkUtils.isSpark2) + Array( + ("rider", 2595), + ("rider,driver", 2735), + ("rider,driver,tip_history", 3750)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + // Test COW / Snapshot + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, "", projectedColumnsReadStats) + } + + @Test + def testMergeOnReadSnapshotRelationWithDeltaLogs(): Unit = { + val tablePath = s"$basePath/mor-with-logs" + val targetRecordsCount = 100 + val targetUpdatedRecordsRatio = 0.5 + + val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount, targetUpdatedRecordsRatio, defaultWriteOpts, populateMetaFields = true) + val tableState = TableState(tablePath, schema, targetRecordsCount, targetUpdatedRecordsRatio) + + // Stats for the reads fetching only _projected_ columns (note how amount of bytes read + // increases along w/ the # of columns) + val projectedColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 2452), + ("rider,driver", 2552), + ("rider,driver,tip_history", 3517)) + else if (HoodieSparkUtils.isSpark2) + Array( + ("rider", 2595), + ("rider,driver", 2735), + ("rider,driver,tip_history", 3750)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + // Stats for the reads fetching _all_ columns (note, how amount of bytes read + // is invariant of the # of columns) + val fullColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 14665), + ("rider,driver", 14665), + ("rider,driver,tip_history", 14665)) + else if (HoodieSparkUtils.isSpark2) + // TODO re-enable tests (these tests are very unstable currently) + Array( + ("rider", -1), + ("rider,driver", -1), + ("rider,driver,tip_history", -1)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + // Test MOR / Snapshot / Skip-merge + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, projectedColumnsReadStats) + + // Test MOR / Snapshot / Payload-combine + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, fullColumnsReadStats) + + // Test MOR / Read Optimized + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, "null", projectedColumnsReadStats) + } + + @Test + def testMergeOnReadSnapshotRelationWithNoDeltaLogs(): Unit = { + val tablePath = s"$basePath/mor-no-logs" + val targetRecordsCount = 100 + val targetUpdatedRecordsRatio = 0.0 + + val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount, targetUpdatedRecordsRatio, defaultWriteOpts, populateMetaFields = true) + val tableState = TableState(tablePath, schema, targetRecordsCount, targetUpdatedRecordsRatio) + + // + // Test #1: MOR table w/ Delta Logs + // + + // Stats for the reads fetching only _projected_ columns (note how amount of bytes read + // increases along w/ the # of columns) + val projectedColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 2452), + ("rider,driver", 2552), + ("rider,driver,tip_history", 3517)) + else if (HoodieSparkUtils.isSpark2) + Array( + ("rider", 2595), + ("rider,driver", 2735), + ("rider,driver,tip_history", 3750)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + // Test MOR / Snapshot / Skip-merge + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, projectedColumnsReadStats) + + // Test MOR / Snapshot / Payload-combine + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL, DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, projectedColumnsReadStats) + + // Test MOR / Read Optimized + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, "null", projectedColumnsReadStats) + } + + // TODO add test for incremental query of the table with logs + @Test + def testMergeOnReadIncrementalRelationWithNoDeltaLogs(): Unit = { + val tablePath = s"$basePath/mor-no-logs" + val targetRecordsCount = 100 + val targetUpdatedRecordsRatio = 0.0 + + val opts: Map[String, String] = + // NOTE: Parquet Compression is disabled as it was leading to non-deterministic outcomes when testing + // against Spark 2.x + defaultWriteOpts ++ Seq(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME.key -> "") + + val (_, schema) = bootstrapMORTable(tablePath, targetRecordsCount, targetUpdatedRecordsRatio, opts, populateMetaFields = true) + val tableState = TableState(tablePath, schema, targetRecordsCount, targetUpdatedRecordsRatio) + + // Stats for the reads fetching only _projected_ columns (note how amount of bytes read + // increases along w/ the # of columns) + val projectedColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 4219), + ("rider,driver", 4279), + ("rider,driver,tip_history", 5186)) + else if (HoodieSparkUtils.isSpark2) + Array( + ("rider", 4430), + ("rider,driver", 4530), + ("rider,driver,tip_history", 5487)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + // Stats for the reads fetching _all_ columns (note, how amount of bytes read + // is invariant of the # of columns) + val fullColumnsReadStats: Array[(String, Long)] = + if (HoodieSparkUtils.isSpark3) + Array( + ("rider", 19683), + ("rider,driver", 19683), + ("rider,driver,tip_history", 19683)) + else if (HoodieSparkUtils.isSpark2) + // TODO re-enable tests (these tests are very unstable currently) + Array( + ("rider", -1), + ("rider,driver", -1), + ("rider,driver,tip_history", -1)) + else + fail("Only Spark 3 and Spark 2 are currently supported") + + val incrementalOpts: Map[String, String] = Map( + DataSourceReadOptions.BEGIN_INSTANTTIME.key -> "001" + ) + + // Test MOR / Incremental / Skip-merge + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL, + projectedColumnsReadStats, incrementalOpts) + + // Test MOR / Incremental / Payload-combine + runTest(tableState, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL, DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL, + fullColumnsReadStats, incrementalOpts) + } + + + // Test routine + private def runTest(tableState: TableState, + queryType: String, + mergeType: String, + expectedStats: Array[(String, Long)], + additionalOpts: Map[String, String] = Map.empty): Unit = { + val tablePath = tableState.path + val readOpts = defaultWriteOpts ++ Map( + "path" -> tablePath, + DataSourceReadOptions.QUERY_TYPE.key -> queryType, + DataSourceReadOptions.REALTIME_MERGE.key -> mergeType + ) ++ additionalOpts + + val ds = new DefaultSource() + val relation: HoodieBaseRelation = ds.createRelation(spark.sqlContext, readOpts).asInstanceOf[HoodieBaseRelation] + + for ((columnListStr, expectedBytesRead) <- expectedStats) { + val targetColumns = columnListStr.split(",") + + println(s"Running test for $tablePath / $queryType / $mergeType / $columnListStr") + + val (rows, bytesRead) = measureBytesRead { () => + val rdd = relation.buildScan(targetColumns, Array.empty).asInstanceOf[HoodieUnsafeRDD] + HoodieUnsafeRDDUtils.collect(rdd) + } + + val targetRecordCount = tableState.targetRecordCount; + val targetUpdatedRecordsRatio = tableState.targetUpdatedRecordsRatio + + val expectedRecordCount = + if (DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL.equals(mergeType)) targetRecordCount * (1 + targetUpdatedRecordsRatio) + else targetRecordCount + + assertEquals(expectedRecordCount, rows.length) + if (expectedBytesRead != -1) { + assertEquals(expectedBytesRead, bytesRead) + } else { + logWarning(s"Not matching bytes read ($bytesRead)") + } + + val readColumns = targetColumns ++ relation.mandatoryColumns + val (_, projectedStructType) = HoodieSparkUtils.getRequiredSchema(tableState.schema, readColumns) + + val row: InternalRow = rows.take(1).head + + // This check is mostly about making sure InternalRow deserializes properly into projected schema + val deserializedColumns = row.toSeq(projectedStructType) + assertEquals(readColumns.length, deserializedColumns.size) + } + } + + private def bootstrapTable(path: String, + tableType: String, + recordCount: Int, + opts: Map[String, String], + populateMetaFields: Boolean, + dataGenOpt: Option[HoodieTestDataGenerator] = None): (List[HoodieRecord[_]], Schema) = { + val dataGen = dataGenOpt.getOrElse(new HoodieTestDataGenerator(0x12345)) + + // Bulk Insert Operation + val schema = + if (populateMetaFields) HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS + else HoodieTestDataGenerator.AVRO_SCHEMA + + val records = dataGen.generateInserts("001", recordCount) + val inputDF: Dataset[Row] = toDataset(records, HoodieTestDataGenerator.AVRO_SCHEMA) + + inputDF.write.format("org.apache.hudi") + .options(opts) + .option(DataSourceWriteOptions.TABLE_TYPE.key, tableType) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(path) + + (records.asScala.toList, schema) + } + + private def bootstrapMORTable(path: String, + recordCount: Int, + updatedRecordsRatio: Double, + opts: Map[String, String], + populateMetaFields: Boolean, + dataGenOpt: Option[HoodieTestDataGenerator] = None): (List[HoodieRecord[_]], Schema) = { + val dataGen = dataGenOpt.getOrElse(new HoodieTestDataGenerator(0x12345)) + + // Step 1: Bootstrap table w/ N records (t/h bulk-insert) + val (insertedRecords, schema) = bootstrapTable(path, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL, recordCount, opts, populateMetaFields, Some(dataGen)) + + if (updatedRecordsRatio == 0) { + (insertedRecords, schema) + } else { + val updatesCount = (insertedRecords.length * updatedRecordsRatio).toInt + val recordsToUpdate = insertedRecords.take(updatesCount) + val updatedRecords = dataGen.generateUpdates("002", recordsToUpdate.asJava) + + // Step 2: Update M records out of those (t/h update) + val inputDF = toDataset(updatedRecords, HoodieTestDataGenerator.AVRO_SCHEMA) + + inputDF.write.format("org.apache.hudi") + .options(opts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(path) + + (updatedRecords.asScala.toList ++ insertedRecords.drop(updatesCount), schema) + } + } + + def measureBytesRead[T](f: () => T): (T, Int) = { + // Init BenchmarkCounter to report number of bytes actually read from the Block + BenchmarkCounter.initCounterFromReporter(HadoopMapRedUtils.createTestReporter, fs.getConf) + val r = f.apply() + val bytesRead = BenchmarkCounter.getBytesRead.toInt + (r, bytesRead) + } + + case class TableState(path: String, schema: Schema, targetRecordCount: Long, targetUpdatedRecordsRatio: Double) +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala new file mode 100644 index 0000000000000..471ebd6107dcc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hudi + +import org.apache.hudi.HoodieSparkUtils +import org.apache.hudi.common.table.HoodieTableMetaClient + +class TestTimeTravelTable extends TestHoodieSqlBase { + test("Test Insert and Update Record with time travel") { + if (HoodieSparkUtils.gteqSpark3_2) { + withTempDir { tmp => + val tableName1 = generateTableName + spark.sql( + s""" + |create table $tableName1 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | tblproperties ( + | type = 'cow', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + | location '${tmp.getCanonicalPath}/$tableName1' + """.stripMargin) + + spark.sql(s"insert into $tableName1 values(1, 'a1', 10, 1000)") + + val metaClient1 = HoodieTableMetaClient.builder() + .setBasePath(s"${tmp.getCanonicalPath}/$tableName1") + .setConf(spark.sessionState.newHadoopConf()) + .build() + + val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline + .lastInstant().get().getTimestamp + + spark.sql(s"insert into $tableName1 values(1, 'a2', 20, 2000)") + + checkAnswer(s"select id, name, price, ts from $tableName1")( + Seq(1, "a2", 20.0, 2000) + ) + + // time travel from instant1 + checkAnswer( + s"select id, name, price, ts from $tableName1 TIMESTAMP AS OF '$instant1'")( + Seq(1, "a1", 10.0, 1000) + ) + } + } + } + + test("Test Insert Into Records with time travel To new Table") { + if (HoodieSparkUtils.gteqSpark3_2) { + withTempDir { tmp => + // Create Non-Partitioned table + val tableName1 = generateTableName + spark.sql( + s""" + |create table $tableName1 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | tblproperties ( + | type = 'cow', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + | location '${tmp.getCanonicalPath}/$tableName1' + """.stripMargin) + + spark.sql(s"insert into $tableName1 values(1, 'a1', 10, 1000)") + + val metaClient1 = HoodieTableMetaClient.builder() + .setBasePath(s"${tmp.getCanonicalPath}/$tableName1") + .setConf(spark.sessionState.newHadoopConf()) + .build() + + val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline + .lastInstant().get().getTimestamp + + + val tableName2 = generateTableName + // Create a partitioned table + spark.sql( + s""" + |create table $tableName2 ( + | id int, + | name string, + | price double, + | ts long, + | dt string + |) using hudi + | tblproperties (primaryKey = 'id') + | partitioned by (dt) + | location '${tmp.getCanonicalPath}/$tableName2' + """.stripMargin) + + // Insert into dynamic partition + spark.sql( + s""" + | insert into $tableName2 + | select id, name, price, ts, '2022-02-14' as dt + | from $tableName1 TIMESTAMP AS OF '$instant1' + """.stripMargin) + checkAnswer(s"select id, name, price, ts, dt from $tableName2")( + Seq(1, "a1", 10.0, 1000, "2022-02-14") + ) + + // Insert into static partition + spark.sql( + s""" + | insert into $tableName2 partition(dt = '2022-02-15') + | select 2 as id, 'a2' as name, price, ts + | from $tableName1 TIMESTAMP AS OF '$instant1' + """.stripMargin) + checkAnswer( + s"select id, name, price, ts, dt from $tableName2")( + Seq(1, "a1", 10.0, 1000, "2022-02-14"), + Seq(2, "a2", 10.0, 1000, "2022-02-15") + ) + } + } + } + + test("Test Two Table's Union Join with time travel") { + if (HoodieSparkUtils.gteqSpark3_2) { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + + val basePath = tmp.getCanonicalPath + val tableName1 = tableName + "_1" + val tableName2 = tableName + "_2" + val path1 = s"$basePath/$tableName1" + val path2 = s"$basePath/$tableName2" + + spark.sql( + s""" + |create table $tableName1 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + | location '$path1' + """.stripMargin) + + spark.sql( + s""" + |create table $tableName2 ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | tblproperties ( + | type = '$tableType', + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + | location '$path2' + """.stripMargin) + + spark.sql(s"insert into $tableName1 values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName1 values(2, 'a2', 20, 1000)") + + checkAnswer(s"select id, name, price, ts from $tableName1")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 20.0, 1000) + ) + + checkAnswer(s"select id, name, price, ts from $tableName1")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 20.0, 1000) + ) + + spark.sql(s"insert into $tableName2 values(3, 'a3', 10, 1000)") + spark.sql(s"insert into $tableName2 values(4, 'a4', 20, 1000)") + + checkAnswer(s"select id, name, price, ts from $tableName2")( + Seq(3, "a3", 10.0, 1000), + Seq(4, "a4", 20.0, 1000) + ) + + val metaClient1 = HoodieTableMetaClient.builder() + .setBasePath(path1) + .setConf(spark.sessionState.newHadoopConf()) + .build() + + val metaClient2 = HoodieTableMetaClient.builder() + .setBasePath(path2) + .setConf(spark.sessionState.newHadoopConf()) + .build() + + val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline + .lastInstant().get().getTimestamp + + val instant2 = metaClient2.getActiveTimeline.getAllCommitsTimeline + .lastInstant().get().getTimestamp + + val sql = + s""" + |select id, name, price, ts from $tableName1 TIMESTAMP AS OF '$instant1' where id=1 + |union + |select id, name, price, ts from $tableName2 TIMESTAMP AS OF '$instant2' where id>1 + |""".stripMargin + + checkAnswer(sql)( + Seq(1, "a1", 10.0, 1000), + Seq(3, "a3", 10.0, 1000), + Seq(4, "a4", 20.0, 1000) + ) + } + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallCommandParser.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala similarity index 98% rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallCommandParser.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala index 9d1c02ad99faa..e26e6617f1871 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallCommandParser.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallCommandParser.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.hudi +package org.apache.spark.sql.hudi.procedure import com.google.common.collect.ImmutableList import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{CallCommand, NamedArgument, PositionalArgument} +import org.apache.spark.sql.hudi.TestHoodieSqlBase import org.apache.spark.sql.types.{DataType, DataTypes} import java.math.BigDecimal diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala similarity index 97% rename from hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallProcedure.scala rename to hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala index eb2c614df201b..bdf4cbe7ba0ff 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestCallProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCallProcedure.scala @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.hudi +package org.apache.spark.sql.hudi.procedure + +import org.apache.spark.sql.hudi.TestHoodieSqlBase class TestCallProcedure extends TestHoodieSqlBase { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunClusteringProcedure.scala new file mode 100644 index 0000000000000..068cd65387057 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunClusteringProcedure.scala @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.procedure + +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline} +import org.apache.hudi.common.util.{Option => HOption} +import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers} +import org.apache.spark.sql.hudi.TestHoodieSqlBase + +import scala.collection.JavaConverters.asScalaIteratorConverter + +class TestRunClusteringProcedure extends TestHoodieSqlBase { + + test("Test Call run_clustering Procedure By Table") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + // Generate the first clustering plan + val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime + client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty()) + + // Generate the second clustering plan + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + val secondScheduleInstant = HoodieActiveTimeline.createNewInstantTime + client.scheduleClusteringAtInstant(secondScheduleInstant, HOption.empty()) + checkAnswer(s"call show_clustering('$tableName')")( + Seq(firstScheduleInstant, 3), + Seq(secondScheduleInstant, 1) + ) + + // Do clustering for all clustering plan generated above, and no new clustering + // instant will be generated because of there is no commit after the second + // clustering plan generated + spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')") + + // No new commits + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, secondScheduleInstant)) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003) + ) + // After clustering there should be no pending clustering. + checkAnswer(s"call show_clustering(table => '$tableName')")() + + // Check the number of finished clustering instants + val finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(2)(finishedClustering.size) + + // Do clustering without manual schedule(which will do the schedule if no pending clustering exists) + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") + spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')") + + val thirdClusteringInstant = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .findInstantsAfter(secondScheduleInstant) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + // Should have a new replace commit after the second clustering command. + assertResult(1)(thirdClusteringInstant.size) + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005) + ) + } + } + } + + test("Test Call run_clustering Procedure By Path") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + + spark.sql(s"call run_clustering(path => '$basePath')") + checkAnswer(s"call show_clustering(path => '$basePath')")() + + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty) + // Generate the first clustering plan + val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime + client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty()) + checkAnswer(s"call show_clustering(path => '$basePath')")( + Seq(firstScheduleInstant, 3) + ) + // Do clustering for all the clustering plan + spark.sql(s"call run_clustering(path => '$basePath', order => 'ts')") + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002) + ) + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + HoodieDataSourceHelpers.hasNewCommits(fs, basePath, firstScheduleInstant) + + // Check the number of finished clustering instants + var finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(1)(finishedClustering.size) + + // Do clustering without manual schedule(which will do the schedule if no pending clustering exists) + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts >= 1003L')") + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004) + ) + + finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(2)(finishedClustering.size) + } + } + } + + test("Test Call run_clustering Procedure With Partition Pruning") { + withTempDir { tmp => + Seq("cow", "mor").foreach { tableType => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | options ( + | primaryKey ='id', + | type = '$tableType', + | preCombineField = 'ts' + | ) + | partitioned by(ts) + | location '$basePath' + """.stripMargin) + + val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf()) + + // Test partition pruning with single predicate + { + spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)") + spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)") + spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)") + + checkException( + s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L and id = 10', order => 'ts')" + )("Only partition predicates are allowed") + + // Do clustering table with partition predicate + spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L', order => 'ts')") + + // There is 1 completed clustering instant + val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(1)(clusteringInstants.size) + + // The latest clustering should contain 2 file groups + val clusteringInstant = clusteringInstants.last + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + assertResult(true)(clusteringPlan.isPresent) + assertResult(2)(clusteringPlan.get().getInputGroups.size()) + + // No pending clustering instant + checkAnswer(s"call show_clustering(table => '$tableName')")() + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002) + ) + } + + // Test partition pruning with {@code And} predicates + { + spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)") + spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)") + spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)") + + checkException( + s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L and id = 10', order => 'ts')" + )("Only partition predicates are allowed") + + // Do clustering table with partition predicate + spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L', order => 'ts')") + + // There are 2 completed clustering instants + val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(2)(clusteringInstants.size) + + // The latest clustering should contain 4 file groups(1002,1003,1004,1005) + val clusteringInstant = clusteringInstants.last + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + assertResult(true)(clusteringPlan.isPresent) + assertResult(4)(clusteringPlan.get().getInputGroups.size()) + + // No pending clustering instant + checkAnswer(s"call show_clustering(table => '$tableName')")() + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005) + ) + } + + // Test partition pruning with {@code And}-{@code Or} predicates + { + spark.sql(s"insert into $tableName values(7, 'a7', 10, 1006)") + spark.sql(s"insert into $tableName values(8, 'a8', 10, 1007)") + spark.sql(s"insert into $tableName values(9, 'a9', 10, 1008)") + spark.sql(s"insert into $tableName values(10, 'a10', 10, 1009)") + + checkException( + s"call run_clustering(table => '$tableName', predicate => 'ts < 1007L or ts >= 1008L or id = 10', order => 'ts')" + )("Only partition predicates are allowed") + + // Do clustering table with partition predicate + spark.sql(s"call run_clustering(table => '$tableName', predicate => '(ts >= 1006L and ts < 1008L) or ts >= 1009L', order => 'ts')") + + // There are 3 completed clustering instants + val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath) + .getInstants + .iterator().asScala + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION) + .toSeq + assertResult(3)(clusteringInstants.size) + + // The latest clustering should contain 3 file groups(1006,1007,1009) + val clusteringInstant = clusteringInstants.last + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + assertResult(true)(clusteringPlan.isPresent) + assertResult(3)(clusteringPlan.get().getInputGroups.size()) + + // No pending clustering instant + checkAnswer(s"call show_clustering(table => '$tableName')")() + + checkAnswer(s"select id, name, price, ts from $tableName order by id")( + Seq(1, "a1", 10.0, 1000), + Seq(2, "a2", 10.0, 1001), + Seq(3, "a3", 10.0, 1002), + Seq(4, "a4", 10.0, 1003), + Seq(5, "a5", 10.0, 1004), + Seq(6, "a6", 10.0, 1005), + Seq(7, "a7", 10.0, 1006), + Seq(8, "a8", 10.0, 1007), + Seq(9, "a9", 10.0, 1008), + Seq(10, "a10", 10.0, 1009) + ) + } + } + } + } +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala new file mode 100644 index 0000000000000..7d60ca018d32a --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestSavepointsProcedure.scala @@ -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.spark.sql.hudi.procedure + +import org.apache.spark.sql.hudi.TestHoodieSqlBase + +class TestSavepointsProcedure extends TestHoodieSqlBase { + + test("Test Call create_savepoints Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + + val commits = spark.sql(s"""call show_commits(table => '$tableName')""").limit(1).collect() + assertResult(1) { + commits.length + } + + val commitTime = commits.apply(0).getString(0) + checkAnswer(s"""call create_savepoints('$tableName', '$commitTime', 'admin', '1')""")(Seq(true)) + } + } + + test("Test Call show_savepoints Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + + val commits = spark.sql(s"""call show_commits(table => '$tableName')""").collect() + assertResult(3) { + commits.length + } + + val commitTime = commits.apply(1).getString(0) + checkAnswer(s"""call create_savepoints('$tableName', '$commitTime')""")(Seq(true)) + + // show savepoints + val savepoints = spark.sql(s"""call show_savepoints(table => '$tableName')""").collect() + assertResult(1) { + savepoints.length + } + } + } + + test("Test Call delete_savepoints Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + spark.sql(s"insert into $tableName select 3, 'a3', 30, 2000") + + val commits = spark.sql(s"""call show_commits(table => '$tableName')""").collect() + assertResult(3) { + commits.length + } + + // create 3 savepoints + commits.foreach(r => { + checkAnswer(s"""call create_savepoints('$tableName', '${r.getString(0)}')""")(Seq(true)) + }) + + // delete savepoints + checkAnswer(s"""call delete_savepoints('$tableName', '${commits.apply(1).getString(0)}')""")(Seq(true)) + + // show savepoints with only 2 + val savepoints = spark.sql(s"""call show_savepoints(table => '$tableName')""").collect() + assertResult(2) { + savepoints.length + } + } + } + + test("Test Call rollback_savepoints Procedure") { + withTempDir { tmp => + val tableName = generateTableName + // create table + spark.sql( + s""" + |create table $tableName ( + | id int, + | name string, + | price double, + | ts long + |) using hudi + | location '${tmp.getCanonicalPath}/$tableName' + | tblproperties ( + | primaryKey = 'id', + | preCombineField = 'ts' + | ) + """.stripMargin) + + // insert data to table + spark.sql(s"insert into $tableName select 1, 'a1', 10, 1000") + spark.sql(s"insert into $tableName select 2, 'a2', 20, 1500") + + val commits = spark.sql(s"""call show_commits(table => '$tableName')""").collect() + assertResult(2) { + commits.length + } + + // create 2 savepoints + commits.foreach(r => { + checkAnswer(s"""call create_savepoints('$tableName', '${r.getString(0)}')""")(Seq(true)) + }) + + // rollback savepoints + checkAnswer(s"""call rollback_savepoints('$tableName', '${commits.apply(0).getString(0)}')""")(Seq(true)) + } + } +} diff --git a/hudi-spark-datasource/hudi-spark2/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 b/hudi-spark-datasource/hudi-spark2/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 index 6544d936243e9..a8b06e4a41709 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 +++ b/hudi-spark-datasource/hudi-spark2/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 @@ -87,6 +87,7 @@ assignmentList assignment : key=qualifiedName EQ value=expression ; + qualifiedNameList : qualifiedName (',' qualifiedName)* ; diff --git a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java index e9ed609ca25f9..4866a5be5c583 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark2/src/main/java/org/apache/hudi/internal/DefaultSource.java @@ -66,7 +66,7 @@ public Optional createWriter(String writeUUID, StructType sche String path = options.get("path").get(); String tblName = options.get(HoodieWriteConfig.TBL_NAME.key()).get(); boolean populateMetaFields = options.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(), - Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())); + HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()); Map properties = options.asMap(); // Auto set the value of "hoodie.parquet.writelegacyformat.enabled" mayBeOverwriteParquetWriteLegacyFormatProp(properties, schema); diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala index 5dfa7d9574d9a..54c8b912a84e6 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.adapter import org.apache.avro.Schema import org.apache.hudi.Spark2RowSerDe import org.apache.hudi.client.utils.SparkRowSerDe -import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializerTrait, Spark2HoodieAvroDeserializer, HoodieAvroSerializer} +import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark2AvroDeserializer, HoodieSparkAvroSerializer} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, Like} @@ -42,11 +42,11 @@ import scala.collection.mutable.ArrayBuffer */ class Spark2Adapter extends SparkAdapter { - def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializerTrait = - new HoodieAvroSerializer(rootCatalystType, rootAvroType, nullable) + def createAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean): HoodieAvroSerializer = + new HoodieSparkAvroSerializer(rootCatalystType, rootAvroType, nullable) - def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializerTrait = - new Spark2HoodieAvroDeserializer(rootAvroType, rootCatalystType) + def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType): HoodieAvroDeserializer = + new HoodieSpark2AvroDeserializer(rootAvroType, rootCatalystType) override def createSparkRowSerDe(encoder: ExpressionEncoder[Row]): SparkRowSerDe = { new Spark2RowSerDe(encoder) @@ -137,4 +137,18 @@ class Spark2Adapter extends SparkAdapter { closePartition() partitions.toSeq } + + /** + * if the logical plan is a TimeTravelRelation LogicalPlan. + */ + override def isRelationTimeTravel(plan: LogicalPlan): Boolean = { + false + } + + /** + * Get the member of the TimeTravelRelation LogicalPlan. + */ + override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = { + throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark2") + } } diff --git a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/HoodieSpark2AvroDeserializer.scala similarity index 76% rename from hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala rename to hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/HoodieSpark2AvroDeserializer.scala index ac2c82f70dacf..2b55c6695e5b2 100644 --- a/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/Spark2HoodieAvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/avro/HoodieSpark2AvroDeserializer.scala @@ -21,13 +21,15 @@ import org.apache.avro.Schema import org.apache.spark.sql.types.DataType /** - * This is Spark 2 implementation for the [[HoodieAvroDeserializerTrait]] leveraging [[PatchedAvroDeserializer]], + * This is Spark 2 implementation for the [[HoodieAvroDeserializer]] leveraging [[PatchedAvroDeserializer]], * which is just copied over version of [[AvroDeserializer]] from Spark 2.4.4 w/ SPARK-30267 being back-ported to it */ -class Spark2HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) - extends HoodieAvroDeserializerTrait { +class HoodieSpark2AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) + extends HoodieAvroDeserializer { private val avroDeserializer = new PatchedAvroDeserializer(rootAvroType, rootCatalystType) - def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data) + // As of Spark 3.1, this will return data wrapped with Option, so we make sure these interfaces + // are aligned across Spark versions + def deserialize(data: Any): Option[Any] = Some(avroDeserializer.deserialize(data)) } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java index 2e46dea390678..4f7ff89a90a33 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/java/org/apache/hudi/spark3/internal/DefaultSource.java @@ -53,7 +53,7 @@ public Table getTable(StructType schema, Transform[] partitioning, Map false } } + + /** + * if the logical plan is a TimeTravelRelation LogicalPlan. + */ + override def isRelationTimeTravel(plan: LogicalPlan): Boolean = { + false + } + + /** + * Get the member of the TimeTravelRelation LogicalPlan. + */ + override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = { + throw new IllegalStateException(s"Should not call getRelationTimeTravel for spark3.1.x") + } } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3AvroDeserializer.scala similarity index 89% rename from hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala rename to hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3AvroDeserializer.scala index fa03f5d841cfb..bd9ead5a70b6d 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/Spark3HoodieAvroDeserializer.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/avro/HoodieSpark3AvroDeserializer.scala @@ -21,8 +21,8 @@ import org.apache.avro.Schema import org.apache.hudi.HoodieSparkUtils import org.apache.spark.sql.types.DataType -class Spark3HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) - extends HoodieAvroDeserializerTrait { +class HoodieSpark3AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) + extends HoodieAvroDeserializer { // SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments. // So use the reflection to get AvroDeserializer instance. @@ -34,5 +34,5 @@ class Spark3HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataT constructor.newInstance(rootAvroType, rootCatalystType) } - def doDeserialize(data: Any): Any = avroDeserializer.deserialize(data) + def deserialize(data: Any): Option[Any] = avroDeserializer.deserialize(data) } diff --git a/hudi-spark-datasource/hudi-spark3/pom.xml b/hudi-spark-datasource/hudi-spark3/pom.xml index 722a1b4101241..d7c8799d4d282 100644 --- a/hudi-spark-datasource/hudi-spark3/pom.xml +++ b/hudi-spark-datasource/hudi-spark3/pom.xml @@ -144,6 +144,24 @@ org.jacoco jacoco-maven-plugin + + org.antlr + antlr4-maven-plugin + ${antlr.version} + + + + antlr4 + + + + + true + true + ../hudi-spark3/src/main/antlr4 + ../hudi-spark3/src/main/antlr4/imports + + diff --git a/hudi-spark-datasource/hudi-spark3/src/main/antlr4/imports/SqlBase.g4 b/hudi-spark-datasource/hudi-spark3/src/main/antlr4/imports/SqlBase.g4 new file mode 100644 index 0000000000000..d4e1e48351ccc --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/antlr4/imports/SqlBase.g4 @@ -0,0 +1,1908 @@ +/* + * Licensed 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. + * + * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar. + */ + +// The parser file is forked from spark 3.2.0's SqlBase.g4. +grammar SqlBase; + +@parser::members { + /** + * When false, INTERSECT is given the greater precedence over the other set + * operations (UNION, EXCEPT and MINUS) as per the SQL standard. + */ + public boolean legacy_setops_precedence_enabled = false; + + /** + * When false, a literal with an exponent would be converted into + * double type rather than decimal type. + */ + public boolean legacy_exponent_literal_as_decimal_enabled = false; + + /** + * When true, the behavior of keywords follows ANSI SQL standard. + */ + public boolean SQL_standard_keyword_behavior = false; +} + +@lexer::members { + /** + * Verify whether current token is a valid decimal token (which contains dot). + * Returns true if the character that follows the token is not a digit or letter or underscore. + * + * For example: + * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. + * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. + * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. + * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is followed + * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' + * which is not a digit or letter or underscore. + */ + public boolean isValidDecimal() { + int nextChar = _input.LA(1); + if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || + nextChar == '_') { + return false; + } else { + return true; + } + } + + /** + * This method will be called when we see '/*' and try to match it as a bracketed comment. + * If the next character is '+', it should be parsed as hint later, and we cannot match + * it as a bracketed comment. + * + * Returns true if the next character is '+'. + */ + public boolean isHint() { + int nextChar = _input.LA(1); + if (nextChar == '+') { + return true; + } else { + return false; + } + } +} + +singleStatement + : statement ';'* EOF + ; + +singleExpression + : namedExpression EOF + ; + +singleTableIdentifier + : tableIdentifier EOF + ; + +singleMultipartIdentifier + : multipartIdentifier EOF + ; + +singleFunctionIdentifier + : functionIdentifier EOF + ; + +singleDataType + : dataType EOF + ; + +singleTableSchema + : colTypeList EOF + ; + +statement + : query #statementDefault + | ctes? dmlStatementNoWith #dmlStatement + | USE NAMESPACE? multipartIdentifier #use + | CREATE namespace (IF NOT EXISTS)? multipartIdentifier + (commentSpec | + locationSpec | + (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace + | ALTER namespace multipartIdentifier + SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties + | ALTER namespace multipartIdentifier + SET locationSpec #setNamespaceLocation + | DROP namespace (IF EXISTS)? multipartIdentifier + (RESTRICT | CASCADE)? #dropNamespace + | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? + (LIKE? pattern=STRING)? #showNamespaces + | createTableHeader ('(' colTypeList ')')? tableProvider? + createTableClauses + (AS? query)? #createTable + | CREATE TABLE (IF NOT EXISTS)? target=tableIdentifier + LIKE source=tableIdentifier + (tableProvider | + rowFormat | + createFileFormat | + locationSpec | + (TBLPROPERTIES tableProps=tablePropertyList))* #createTableLike + | replaceTableHeader ('(' colTypeList ')')? tableProvider? + createTableClauses + (AS? query)? #replaceTable + | ANALYZE TABLE multipartIdentifier partitionSpec? COMPUTE STATISTICS + (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze + | ANALYZE TABLES ((FROM | IN) multipartIdentifier)? COMPUTE STATISTICS + (identifier)? #analyzeTables + | ALTER TABLE multipartIdentifier + ADD (COLUMN | COLUMNS) + columns=qualifiedColTypeWithPositionList #addTableColumns + | ALTER TABLE multipartIdentifier + ADD (COLUMN | COLUMNS) + '(' columns=qualifiedColTypeWithPositionList ')' #addTableColumns + | ALTER TABLE table=multipartIdentifier + RENAME COLUMN + from=multipartIdentifier TO to=errorCapturingIdentifier #renameTableColumn + | ALTER TABLE multipartIdentifier + DROP (COLUMN | COLUMNS) + '(' columns=multipartIdentifierList ')' #dropTableColumns + | ALTER TABLE multipartIdentifier + DROP (COLUMN | COLUMNS) columns=multipartIdentifierList #dropTableColumns + | ALTER (TABLE | VIEW) from=multipartIdentifier + RENAME TO to=multipartIdentifier #renameTable + | ALTER (TABLE | VIEW) multipartIdentifier + SET TBLPROPERTIES tablePropertyList #setTableProperties + | ALTER (TABLE | VIEW) multipartIdentifier + UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties + | ALTER TABLE table=multipartIdentifier + (ALTER | CHANGE) COLUMN? column=multipartIdentifier + alterColumnAction? #alterTableAlterColumn + | ALTER TABLE table=multipartIdentifier partitionSpec? + CHANGE COLUMN? + colName=multipartIdentifier colType colPosition? #hiveChangeColumn + | ALTER TABLE table=multipartIdentifier partitionSpec? + REPLACE COLUMNS + '(' columns=qualifiedColTypeWithPositionList ')' #hiveReplaceColumns + | ALTER TABLE multipartIdentifier (partitionSpec)? + SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe + | ALTER TABLE multipartIdentifier (partitionSpec)? + SET SERDEPROPERTIES tablePropertyList #setTableSerDe + | ALTER (TABLE | VIEW) multipartIdentifier ADD (IF NOT EXISTS)? + partitionSpecLocation+ #addTablePartition + | ALTER TABLE multipartIdentifier + from=partitionSpec RENAME TO to=partitionSpec #renameTablePartition + | ALTER (TABLE | VIEW) multipartIdentifier + DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions + | ALTER TABLE multipartIdentifier + (partitionSpec)? SET locationSpec #setTableLocation + | ALTER TABLE multipartIdentifier RECOVER PARTITIONS #recoverPartitions + | DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable + | DROP VIEW (IF EXISTS)? multipartIdentifier #dropView + | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? + VIEW (IF NOT EXISTS)? multipartIdentifier + identifierCommentList? + (commentSpec | + (PARTITIONED ON identifierList) | + (TBLPROPERTIES tablePropertyList))* + AS query #createView + | CREATE (OR REPLACE)? GLOBAL? TEMPORARY VIEW + tableIdentifier ('(' colTypeList ')')? tableProvider + (OPTIONS tablePropertyList)? #createTempViewUsing + | ALTER VIEW multipartIdentifier AS? query #alterViewQuery + | CREATE (OR REPLACE)? TEMPORARY? FUNCTION (IF NOT EXISTS)? + multipartIdentifier AS className=STRING + (USING resource (',' resource)*)? #createFunction + | DROP TEMPORARY? FUNCTION (IF EXISTS)? multipartIdentifier #dropFunction + | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? + statement #explain + | SHOW TABLES ((FROM | IN) multipartIdentifier)? + (LIKE? pattern=STRING)? #showTables + | SHOW TABLE EXTENDED ((FROM | IN) ns=multipartIdentifier)? + LIKE pattern=STRING partitionSpec? #showTableExtended + | SHOW TBLPROPERTIES table=multipartIdentifier + ('(' key=tablePropertyKey ')')? #showTblProperties + | SHOW COLUMNS (FROM | IN) table=multipartIdentifier + ((FROM | IN) ns=multipartIdentifier)? #showColumns + | SHOW VIEWS ((FROM | IN) multipartIdentifier)? + (LIKE? pattern=STRING)? #showViews + | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions + | SHOW identifier? FUNCTIONS + (LIKE? (multipartIdentifier | pattern=STRING))? #showFunctions + | SHOW CREATE TABLE multipartIdentifier (AS SERDE)? #showCreateTable + | SHOW CURRENT NAMESPACE #showCurrentNamespace + | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction + | (DESC | DESCRIBE) namespace EXTENDED? + multipartIdentifier #describeNamespace + | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? + multipartIdentifier partitionSpec? describeColName? #describeRelation + | (DESC | DESCRIBE) QUERY? query #describeQuery + | COMMENT ON namespace multipartIdentifier IS + comment=(STRING | NULL) #commentNamespace + | COMMENT ON TABLE multipartIdentifier IS comment=(STRING | NULL) #commentTable + | REFRESH TABLE multipartIdentifier #refreshTable + | REFRESH FUNCTION multipartIdentifier #refreshFunction + | REFRESH (STRING | .*?) #refreshResource + | CACHE LAZY? TABLE multipartIdentifier + (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable + | UNCACHE TABLE (IF EXISTS)? multipartIdentifier #uncacheTable + | CLEAR CACHE #clearCache + | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE + multipartIdentifier partitionSpec? #loadData + | TRUNCATE TABLE multipartIdentifier partitionSpec? #truncateTable + | MSCK REPAIR TABLE multipartIdentifier + (option=(ADD|DROP|SYNC) PARTITIONS)? #repairTable + | op=(ADD | LIST) identifier .*? #manageResource + | SET ROLE .*? #failNativeCommand + | SET TIME ZONE interval #setTimeZone + | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone + | SET TIME ZONE .*? #setTimeZone + | SET configKey EQ configValue #setQuotedConfiguration + | SET configKey (EQ .*?)? #setQuotedConfiguration + | SET .*? EQ configValue #setQuotedConfiguration + | SET .*? #setConfiguration + | RESET configKey #resetQuotedConfiguration + | RESET .*? #resetConfiguration + | unsupportedHiveNativeCommands .*? #failNativeCommand + ; + +configKey + : quotedIdentifier + ; + +configValue + : quotedIdentifier + ; + +unsupportedHiveNativeCommands + : kw1=CREATE kw2=ROLE + | kw1=DROP kw2=ROLE + | kw1=GRANT kw2=ROLE? + | kw1=REVOKE kw2=ROLE? + | kw1=SHOW kw2=GRANT + | kw1=SHOW kw2=ROLE kw3=GRANT? + | kw1=SHOW kw2=PRINCIPALS + | kw1=SHOW kw2=ROLES + | kw1=SHOW kw2=CURRENT kw3=ROLES + | kw1=EXPORT kw2=TABLE + | kw1=IMPORT kw2=TABLE + | kw1=SHOW kw2=COMPACTIONS + | kw1=SHOW kw2=CREATE kw3=TABLE + | kw1=SHOW kw2=TRANSACTIONS + | kw1=SHOW kw2=INDEXES + | kw1=SHOW kw2=LOCKS + | kw1=CREATE kw2=INDEX + | kw1=DROP kw2=INDEX + | kw1=ALTER kw2=INDEX + | kw1=LOCK kw2=TABLE + | kw1=LOCK kw2=DATABASE + | kw1=UNLOCK kw2=TABLE + | kw1=UNLOCK kw2=DATABASE + | kw1=CREATE kw2=TEMPORARY kw3=MACRO + | kw1=DROP kw2=TEMPORARY kw3=MACRO + | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=CLUSTERED + | kw1=ALTER kw2=TABLE tableIdentifier kw3=CLUSTERED kw4=BY + | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=SORTED + | kw1=ALTER kw2=TABLE tableIdentifier kw3=SKEWED kw4=BY + | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=SKEWED + | kw1=ALTER kw2=TABLE tableIdentifier kw3=NOT kw4=STORED kw5=AS kw6=DIRECTORIES + | kw1=ALTER kw2=TABLE tableIdentifier kw3=SET kw4=SKEWED kw5=LOCATION + | kw1=ALTER kw2=TABLE tableIdentifier kw3=EXCHANGE kw4=PARTITION + | kw1=ALTER kw2=TABLE tableIdentifier kw3=ARCHIVE kw4=PARTITION + | kw1=ALTER kw2=TABLE tableIdentifier kw3=UNARCHIVE kw4=PARTITION + | kw1=ALTER kw2=TABLE tableIdentifier kw3=TOUCH + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=COMPACT + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=CONCATENATE + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=SET kw4=FILEFORMAT + | kw1=ALTER kw2=TABLE tableIdentifier partitionSpec? kw3=REPLACE kw4=COLUMNS + | kw1=START kw2=TRANSACTION + | kw1=COMMIT + | kw1=ROLLBACK + | kw1=DFS + ; + +createTableHeader + : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier + ; + +replaceTableHeader + : (CREATE OR)? REPLACE TABLE multipartIdentifier + ; + +bucketSpec + : CLUSTERED BY identifierList + (SORTED BY orderedIdentifierList)? + INTO INTEGER_VALUE BUCKETS + ; + +skewSpec + : SKEWED BY identifierList + ON (constantList | nestedConstantList) + (STORED AS DIRECTORIES)? + ; + +locationSpec + : LOCATION STRING + ; + +commentSpec + : COMMENT STRING + ; + +query + : ctes? queryTerm queryOrganization + ; + +insertInto + : INSERT OVERWRITE TABLE? multipartIdentifier (partitionSpec (IF NOT EXISTS)?)? identifierList? #insertOverwriteTable + | INSERT INTO TABLE? multipartIdentifier partitionSpec? (IF NOT EXISTS)? identifierList? #insertIntoTable + | INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir + ; + +partitionSpecLocation + : partitionSpec locationSpec? + ; + +partitionSpec + : PARTITION '(' partitionVal (',' partitionVal)* ')' + ; + +partitionVal + : identifier (EQ constant)? + ; + +namespace + : NAMESPACE + | DATABASE + | SCHEMA + ; + +describeFuncName + : qualifiedName + | STRING + | comparisonOperator + | arithmeticOperator + | predicateOperator + ; + +describeColName + : nameParts+=identifier ('.' nameParts+=identifier)* + ; + +ctes + : WITH namedQuery (',' namedQuery)* + ; + +namedQuery + : name=errorCapturingIdentifier (columnAliases=identifierList)? AS? '(' query ')' + ; + +tableProvider + : USING multipartIdentifier + ; + +createTableClauses + :((OPTIONS options=tablePropertyList) | + (PARTITIONED BY partitioning=partitionFieldList) | + skewSpec | + bucketSpec | + rowFormat | + createFileFormat | + locationSpec | + commentSpec | + (TBLPROPERTIES tableProps=tablePropertyList))* + ; + +tablePropertyList + : '(' tableProperty (',' tableProperty)* ')' + ; + +tableProperty + : key=tablePropertyKey (EQ? value=tablePropertyValue)? + ; + +tablePropertyKey + : identifier ('.' identifier)* + | STRING + ; + +tablePropertyValue + : INTEGER_VALUE + | DECIMAL_VALUE + | booleanValue + | STRING + ; + +constantList + : '(' constant (',' constant)* ')' + ; + +nestedConstantList + : '(' constantList (',' constantList)* ')' + ; + +createFileFormat + : STORED AS fileFormat + | STORED BY storageHandler + ; + +fileFormat + : INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING #tableFileFormat + | identifier #genericFileFormat + ; + +storageHandler + : STRING (WITH SERDEPROPERTIES tablePropertyList)? + ; + +resource + : identifier STRING + ; + +dmlStatementNoWith + : insertInto queryTerm queryOrganization #singleInsertQuery + | fromClause multiInsertQueryBody+ #multiInsertQuery + | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable + | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable + | MERGE INTO target=multipartIdentifier targetAlias=tableAlias + USING (source=multipartIdentifier | + '(' sourceQuery=query')') sourceAlias=tableAlias + ON mergeCondition=booleanExpression + matchedClause* + notMatchedClause* #mergeIntoTable + ; + +queryOrganization + : (ORDER BY order+=sortItem (',' order+=sortItem)*)? + (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)? + (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)? + (SORT BY sort+=sortItem (',' sort+=sortItem)*)? + windowClause? + (LIMIT (ALL | limit=expression))? + ; + +multiInsertQueryBody + : insertInto fromStatementBody + ; + +queryTerm + : queryPrimary #queryTermDefault + | left=queryTerm {legacy_setops_precedence_enabled}? + operator=(INTERSECT | UNION | EXCEPT | SETMINUS) setQuantifier? right=queryTerm #setOperation + | left=queryTerm {!legacy_setops_precedence_enabled}? + operator=INTERSECT setQuantifier? right=queryTerm #setOperation + | left=queryTerm {!legacy_setops_precedence_enabled}? + operator=(UNION | EXCEPT | SETMINUS) setQuantifier? right=queryTerm #setOperation + ; + +queryPrimary + : querySpecification #queryPrimaryDefault + | fromStatement #fromStmt + | TABLE multipartIdentifier #table + | inlineTable #inlineTableDefault1 + | '(' query ')' #subquery + ; + +sortItem + : expression ordering=(ASC | DESC)? (NULLS nullOrder=(LAST | FIRST))? + ; + +fromStatement + : fromClause fromStatementBody+ + ; + +fromStatementBody + : transformClause + whereClause? + queryOrganization + | selectClause + lateralView* + whereClause? + aggregationClause? + havingClause? + windowClause? + queryOrganization + ; + +querySpecification + : transformClause + fromClause? + lateralView* + whereClause? + aggregationClause? + havingClause? + windowClause? #transformQuerySpecification + | selectClause + fromClause? + lateralView* + whereClause? + aggregationClause? + havingClause? + windowClause? #regularQuerySpecification + ; + +transformClause + : (SELECT kind=TRANSFORM '(' setQuantifier? expressionSeq ')' + | kind=MAP setQuantifier? expressionSeq + | kind=REDUCE setQuantifier? expressionSeq) + inRowFormat=rowFormat? + (RECORDWRITER recordWriter=STRING)? + USING script=STRING + (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))? + outRowFormat=rowFormat? + (RECORDREADER recordReader=STRING)? + ; + +selectClause + : SELECT (hints+=hint)* setQuantifier? namedExpressionSeq + ; + +setClause + : SET assignmentList + ; + +matchedClause + : WHEN MATCHED (AND matchedCond=booleanExpression)? THEN matchedAction + ; +notMatchedClause + : WHEN NOT MATCHED (AND notMatchedCond=booleanExpression)? THEN notMatchedAction + ; + +matchedAction + : DELETE + | UPDATE SET ASTERISK + | UPDATE SET assignmentList + ; + +notMatchedAction + : INSERT ASTERISK + | INSERT '(' columns=multipartIdentifierList ')' + VALUES '(' expression (',' expression)* ')' + ; + +assignmentList + : assignment (',' assignment)* + ; + +assignment + : key=multipartIdentifier EQ value=expression + ; + +whereClause + : WHERE booleanExpression + ; + +havingClause + : HAVING booleanExpression + ; + +hint + : '/*+' hintStatements+=hintStatement (','? hintStatements+=hintStatement)* '*/' + ; + +hintStatement + : hintName=identifier + | hintName=identifier '(' parameters+=primaryExpression (',' parameters+=primaryExpression)* ')' + ; + +fromClause + : FROM relation (',' relation)* lateralView* pivotClause? + ; + +temporalClause + : FOR? (SYSTEM_TIME | TIMESTAMP) AS OF timestamp=valueExpression + | FOR? (SYSTEM_VERSION | VERSION) AS OF version=(INTEGER_VALUE | STRING) + ; + +aggregationClause + : GROUP BY groupingExpressionsWithGroupingAnalytics+=groupByClause + (',' groupingExpressionsWithGroupingAnalytics+=groupByClause)* + | GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* ( + WITH kind=ROLLUP + | WITH kind=CUBE + | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')? + ; + +groupByClause + : groupingAnalytics + | expression + ; + +groupingAnalytics + : (ROLLUP | CUBE) '(' groupingSet (',' groupingSet)* ')' + | GROUPING SETS '(' groupingElement (',' groupingElement)* ')' + ; + +groupingElement + : groupingAnalytics + | groupingSet + ; + +groupingSet + : '(' (expression (',' expression)*)? ')' + | expression + ; + +pivotClause + : PIVOT '(' aggregates=namedExpressionSeq FOR pivotColumn IN '(' pivotValues+=pivotValue (',' pivotValues+=pivotValue)* ')' ')' + ; + +pivotColumn + : identifiers+=identifier + | '(' identifiers+=identifier (',' identifiers+=identifier)* ')' + ; + +pivotValue + : expression (AS? identifier)? + ; + +lateralView + : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)? + ; + +setQuantifier + : DISTINCT + | ALL + ; + +relation + : LATERAL? relationPrimary joinRelation* + ; + +joinRelation + : (joinType) JOIN LATERAL? right=relationPrimary joinCriteria? + | NATURAL joinType JOIN LATERAL? right=relationPrimary + ; + +joinType + : INNER? + | CROSS + | LEFT OUTER? + | LEFT? SEMI + | RIGHT OUTER? + | FULL OUTER? + | LEFT? ANTI + ; + +joinCriteria + : ON booleanExpression + | USING identifierList + ; + +sample + : TABLESAMPLE '(' sampleMethod? ')' + ; + +sampleMethod + : negativeSign=MINUS? percentage=(INTEGER_VALUE | DECIMAL_VALUE) PERCENTLIT #sampleByPercentile + | expression ROWS #sampleByRows + | sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE + (ON (identifier | qualifiedName '(' ')'))? #sampleByBucket + | bytes=expression #sampleByBytes + ; + +identifierList + : '(' identifierSeq ')' + ; + +identifierSeq + : ident+=errorCapturingIdentifier (',' ident+=errorCapturingIdentifier)* + ; + +orderedIdentifierList + : '(' orderedIdentifier (',' orderedIdentifier)* ')' + ; + +orderedIdentifier + : ident=errorCapturingIdentifier ordering=(ASC | DESC)? + ; + +identifierCommentList + : '(' identifierComment (',' identifierComment)* ')' + ; + +identifierComment + : identifier commentSpec? + ; + +relationPrimary + : multipartIdentifier temporalClause? + sample? tableAlias #tableName + | '(' query ')' sample? tableAlias #aliasedQuery + | '(' relation ')' sample? tableAlias #aliasedRelation + | inlineTable #inlineTableDefault2 + | functionTable #tableValuedFunction + ; + +inlineTable + : VALUES expression (',' expression)* tableAlias + ; + +functionTable + : funcName=functionName '(' (expression (',' expression)*)? ')' tableAlias + ; + +tableAlias + : (AS? strictIdentifier identifierList?)? + ; + +rowFormat + : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tablePropertyList)? #rowFormatSerde + | ROW FORMAT DELIMITED + (FIELDS TERMINATED BY fieldsTerminatedBy=STRING (ESCAPED BY escapedBy=STRING)?)? + (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)? + (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)? + (LINES TERMINATED BY linesSeparatedBy=STRING)? + (NULL DEFINED AS nullDefinedAs=STRING)? #rowFormatDelimited + ; + +multipartIdentifierList + : multipartIdentifier (',' multipartIdentifier)* + ; + +multipartIdentifier + : parts+=errorCapturingIdentifier ('.' parts+=errorCapturingIdentifier)* + ; + +tableIdentifier + : (db=errorCapturingIdentifier '.')? table=errorCapturingIdentifier + ; + +functionIdentifier + : (db=errorCapturingIdentifier '.')? function=errorCapturingIdentifier + ; + +namedExpression + : expression (AS? (name=errorCapturingIdentifier | identifierList))? + ; + +namedExpressionSeq + : namedExpression (',' namedExpression)* + ; + +partitionFieldList + : '(' fields+=partitionField (',' fields+=partitionField)* ')' + ; + +partitionField + : transform #partitionTransform + | colType #partitionColumn + ; + +transform + : qualifiedName #identityTransform + | transformName=identifier + '(' argument+=transformArgument (',' argument+=transformArgument)* ')' #applyTransform + ; + +transformArgument + : qualifiedName + | constant + ; + +expression + : booleanExpression + ; + +expressionSeq + : expression (',' expression)* + ; + +booleanExpression + : NOT booleanExpression #logicalNot + | EXISTS '(' query ')' #exists + | valueExpression predicate? #predicated + | left=booleanExpression operator=AND right=booleanExpression #logicalBinary + | left=booleanExpression operator=OR right=booleanExpression #logicalBinary + ; + +predicate + : NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression + | NOT? kind=IN '(' expression (',' expression)* ')' + | NOT? kind=IN '(' query ')' + | NOT? kind=RLIKE pattern=valueExpression + | NOT? kind=LIKE quantifier=(ANY | SOME | ALL) ('('')' | '(' expression (',' expression)* ')') + | NOT? kind=LIKE pattern=valueExpression (ESCAPE escapeChar=STRING)? + | IS NOT? kind=NULL + | IS NOT? kind=(TRUE | FALSE | UNKNOWN) + | IS NOT? kind=DISTINCT FROM right=valueExpression + ; + +valueExpression + : primaryExpression #valueExpressionDefault + | operator=(MINUS | PLUS | TILDE) valueExpression #arithmeticUnary + | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary + | left=valueExpression operator=(PLUS | MINUS | CONCAT_PIPE) right=valueExpression #arithmeticBinary + | left=valueExpression operator=AMPERSAND right=valueExpression #arithmeticBinary + | left=valueExpression operator=HAT right=valueExpression #arithmeticBinary + | left=valueExpression operator=PIPE right=valueExpression #arithmeticBinary + | left=valueExpression comparisonOperator right=valueExpression #comparison + ; + +primaryExpression + : name=(CURRENT_DATE | CURRENT_TIMESTAMP | CURRENT_USER) #currentLike + | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase + | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase + | name=(CAST | TRY_CAST) '(' expression AS dataType ')' #cast + | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct + | FIRST '(' expression (IGNORE NULLS)? ')' #first + | LAST '(' expression (IGNORE NULLS)? ')' #last + | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position + | constant #constantDefault + | ASTERISK #star + | qualifiedName '.' ASTERISK #star + | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor + | '(' query ')' #subqueryExpression + | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' + (FILTER '(' WHERE where=booleanExpression ')')? + (nullsOption=(IGNORE | RESPECT) NULLS)? ( OVER windowSpec)? #functionCall + | identifier '->' expression #lambda + | '(' identifier (',' identifier)+ ')' '->' expression #lambda + | value=primaryExpression '[' index=valueExpression ']' #subscript + | identifier #columnReference + | base=primaryExpression '.' fieldName=identifier #dereference + | '(' expression ')' #parenthesizedExpression + | EXTRACT '(' field=identifier FROM source=valueExpression ')' #extract + | (SUBSTR | SUBSTRING) '(' str=valueExpression (FROM | ',') pos=valueExpression + ((FOR | ',') len=valueExpression)? ')' #substring + | TRIM '(' trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)? + FROM srcStr=valueExpression ')' #trim + | OVERLAY '(' input=valueExpression PLACING replace=valueExpression + FROM position=valueExpression (FOR length=valueExpression)? ')' #overlay + ; + +constant + : NULL #nullLiteral + | interval #intervalLiteral + | identifier STRING #typeConstructor + | number #numericLiteral + | booleanValue #booleanLiteral + | STRING+ #stringLiteral + ; + +comparisonOperator + : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ + ; + +arithmeticOperator + : PLUS | MINUS | ASTERISK | SLASH | PERCENT | DIV | TILDE | AMPERSAND | PIPE | CONCAT_PIPE | HAT + ; + +predicateOperator + : OR | AND | IN | NOT + ; + +booleanValue + : TRUE | FALSE + ; + +interval + : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? + ; + +errorCapturingMultiUnitsInterval + : body=multiUnitsInterval unitToUnitInterval? + ; + +multiUnitsInterval + : (intervalValue unit+=identifier)+ + ; + +errorCapturingUnitToUnitInterval + : body=unitToUnitInterval (error1=multiUnitsInterval | error2=unitToUnitInterval)? + ; + +unitToUnitInterval + : value=intervalValue from=identifier TO to=identifier + ; + +intervalValue + : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE | STRING) + ; + +colPosition + : position=FIRST | position=AFTER afterCol=errorCapturingIdentifier + ; + +dataType + : complex=ARRAY '<' dataType '>' #complexDataType + | complex=MAP '<' dataType ',' dataType '>' #complexDataType + | complex=STRUCT ('<' complexColTypeList? '>' | NEQ) #complexDataType + | INTERVAL from=(YEAR | MONTH) (TO to=MONTH)? #yearMonthIntervalDataType + | INTERVAL from=(DAY | HOUR | MINUTE | SECOND) + (TO to=(HOUR | MINUTE | SECOND))? #dayTimeIntervalDataType + | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')? #primitiveDataType + ; + +qualifiedColTypeWithPositionList + : qualifiedColTypeWithPosition (',' qualifiedColTypeWithPosition)* + ; + +qualifiedColTypeWithPosition + : name=multipartIdentifier dataType (NOT NULL)? commentSpec? colPosition? + ; + +colTypeList + : colType (',' colType)* + ; + +colType + : colName=errorCapturingIdentifier dataType (NOT NULL)? commentSpec? + ; + +complexColTypeList + : complexColType (',' complexColType)* + ; + +complexColType + : identifier ':'? dataType (NOT NULL)? commentSpec? + ; + +whenClause + : WHEN condition=expression THEN result=expression + ; + +windowClause + : WINDOW namedWindow (',' namedWindow)* + ; + +namedWindow + : name=errorCapturingIdentifier AS windowSpec + ; + +windowSpec + : name=errorCapturingIdentifier #windowRef + | '('name=errorCapturingIdentifier')' #windowRef + | '(' + ( CLUSTER BY partition+=expression (',' partition+=expression)* + | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)? + ((ORDER | SORT) BY sortItem (',' sortItem)*)?) + windowFrame? + ')' #windowDef + ; + +windowFrame + : frameType=RANGE start=frameBound + | frameType=ROWS start=frameBound + | frameType=RANGE BETWEEN start=frameBound AND end=frameBound + | frameType=ROWS BETWEEN start=frameBound AND end=frameBound + ; + +frameBound + : UNBOUNDED boundType=(PRECEDING | FOLLOWING) + | boundType=CURRENT ROW + | expression boundType=(PRECEDING | FOLLOWING) + ; + +qualifiedNameList + : qualifiedName (',' qualifiedName)* + ; + +functionName + : qualifiedName + | FILTER + | LEFT + | RIGHT + ; + +qualifiedName + : identifier ('.' identifier)* + ; + +// this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table` +// replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise +// valid expressions such as "a-b" can be recognized as an identifier +errorCapturingIdentifier + : identifier errorCapturingIdentifierExtra + ; + +// extra left-factoring grammar +errorCapturingIdentifierExtra + : (MINUS identifier)+ #errorIdent + | #realIdent + ; + +identifier + : strictIdentifier + | {!SQL_standard_keyword_behavior}? strictNonReserved + ; + +strictIdentifier + : IDENTIFIER #unquotedIdentifier + | quotedIdentifier #quotedIdentifierAlternative + | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier + ; + +quotedIdentifier + : BACKQUOTED_IDENTIFIER + ; + +number + : {!legacy_exponent_literal_as_decimal_enabled}? MINUS? EXPONENT_VALUE #exponentLiteral + | {!legacy_exponent_literal_as_decimal_enabled}? MINUS? DECIMAL_VALUE #decimalLiteral + | {legacy_exponent_literal_as_decimal_enabled}? MINUS? (EXPONENT_VALUE | DECIMAL_VALUE) #legacyDecimalLiteral + | MINUS? INTEGER_VALUE #integerLiteral + | MINUS? BIGINT_LITERAL #bigIntLiteral + | MINUS? SMALLINT_LITERAL #smallIntLiteral + | MINUS? TINYINT_LITERAL #tinyIntLiteral + | MINUS? DOUBLE_LITERAL #doubleLiteral + | MINUS? FLOAT_LITERAL #floatLiteral + | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral + ; + +alterColumnAction + : TYPE dataType + | commentSpec + | colPosition + | setOrDrop=(SET | DROP) NOT NULL + ; + +// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. +// - Reserved keywords: +// Keywords that are reserved and can't be used as identifiers for table, view, column, +// function, alias, etc. +// - Non-reserved keywords: +// Keywords that have a special meaning only in particular contexts and can be used as +// identifiers in other contexts. For example, `EXPLAIN SELECT ...` is a command, but EXPLAIN +// can be used as identifiers in other places. +// You can find the full keywords list by searching "Start of the keywords list" in this file. +// The non-reserved keywords are listed below. Keywords not in this list are reserved keywords. +ansiNonReserved +//--ANSI-NON-RESERVED-START + : ADD + | AFTER + | ALTER + | ANALYZE + | ANTI + | ARCHIVE + | ARRAY + | ASC + | AT + | BETWEEN + | BUCKET + | BUCKETS + | BY + | CACHE + | CASCADE + | CHANGE + | CLEAR + | CLUSTER + | CLUSTERED + | CODEGEN + | COLLECTION + | COLUMNS + | COMMENT + | COMMIT + | COMPACT + | COMPACTIONS + | COMPUTE + | CONCATENATE + | COST + | CUBE + | CURRENT + | DATA + | DATABASE + | DATABASES + | DAY + | DBPROPERTIES + | DEFINED + | DELETE + | DELIMITED + | DESC + | DESCRIBE + | DFS + | DIRECTORIES + | DIRECTORY + | DISTRIBUTE + | DIV + | DROP + | ESCAPED + | EXCHANGE + | EXISTS + | EXPLAIN + | EXPORT + | EXTENDED + | EXTERNAL + | EXTRACT + | FIELDS + | FILEFORMAT + | FIRST + | FOLLOWING + | FORMAT + | FORMATTED + | FUNCTION + | FUNCTIONS + | GLOBAL + | GROUPING + | HOUR + | IF + | IGNORE + | IMPORT + | INDEX + | INDEXES + | INPATH + | INPUTFORMAT + | INSERT + | INTERVAL + | ITEMS + | KEYS + | LAST + | LAZY + | LIKE + | LIMIT + | LINES + | LIST + | LOAD + | LOCAL + | LOCATION + | LOCK + | LOCKS + | LOGICAL + | MACRO + | MAP + | MATCHED + | MERGE + | MINUTE + | MONTH + | MSCK + | NAMESPACE + | NAMESPACES + | NO + | NULLS + | OF + | OPTION + | OPTIONS + | OUT + | OUTPUTFORMAT + | OVER + | OVERLAY + | OVERWRITE + | PARTITION + | PARTITIONED + | PARTITIONS + | PERCENTLIT + | PIVOT + | PLACING + | POSITION + | PRECEDING + | PRINCIPALS + | PROPERTIES + | PURGE + | QUERY + | RANGE + | RECORDREADER + | RECORDWRITER + | RECOVER + | REDUCE + | REFRESH + | RENAME + | REPAIR + | REPLACE + | RESET + | RESPECT + | RESTRICT + | REVOKE + | RLIKE + | ROLE + | ROLES + | ROLLBACK + | ROLLUP + | ROW + | ROWS + | SCHEMA + | SECOND + | SEMI + | SEPARATED + | SERDE + | SERDEPROPERTIES + | SET + | SETMINUS + | SETS + | SHOW + | SKEWED + | SORT + | SORTED + | START + | STATISTICS + | STORED + | STRATIFY + | STRUCT + | SUBSTR + | SUBSTRING + | SYNC + | TABLES + | TABLESAMPLE + | TBLPROPERTIES + | TEMPORARY + | TERMINATED + | TOUCH + | TRANSACTION + | TRANSACTIONS + | TRANSFORM + | TRIM + | TRUE + | TRUNCATE + | TRY_CAST + | TYPE + | UNARCHIVE + | UNBOUNDED + | UNCACHE + | UNLOCK + | UNSET + | UPDATE + | USE + | VALUES + | VIEW + | VIEWS + | WINDOW + | YEAR + | ZONE +//--ANSI-NON-RESERVED-END + ; + +// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. +// - Non-reserved keywords: +// Same definition as the one when `SQL_standard_keyword_behavior=true`. +// - Strict-non-reserved keywords: +// A strict version of non-reserved keywords, which can not be used as table alias. +// You can find the full keywords list by searching "Start of the keywords list" in this file. +// The strict-non-reserved keywords are listed in `strictNonReserved`. +// The non-reserved keywords are listed in `nonReserved`. +// These 2 together contain all the keywords. +strictNonReserved + : ANTI + | CROSS + | EXCEPT + | FULL + | INNER + | INTERSECT + | JOIN + | LATERAL + | LEFT + | NATURAL + | ON + | RIGHT + | SEMI + | SETMINUS + | UNION + | USING + ; + +nonReserved +//--DEFAULT-NON-RESERVED-START + : ADD + | AFTER + | ALL + | ALTER + | ANALYZE + | AND + | ANY + | ARCHIVE + | ARRAY + | AS + | ASC + | AT + | AUTHORIZATION + | BETWEEN + | BOTH + | BUCKET + | BUCKETS + | BY + | CACHE + | CASCADE + | CASE + | CAST + | CHANGE + | CHECK + | CLEAR + | CLUSTER + | CLUSTERED + | CODEGEN + | COLLATE + | COLLECTION + | COLUMN + | COLUMNS + | COMMENT + | COMMIT + | COMPACT + | COMPACTIONS + | COMPUTE + | CONCATENATE + | CONSTRAINT + | COST + | CREATE + | CUBE + | CURRENT + | CURRENT_DATE + | CURRENT_TIME + | CURRENT_TIMESTAMP + | CURRENT_USER + | DATA + | DATABASE + | DATABASES + | DAY + | DBPROPERTIES + | DEFINED + | DELETE + | DELIMITED + | DESC + | DESCRIBE + | DFS + | DIRECTORIES + | DIRECTORY + | DISTINCT + | DISTRIBUTE + | DIV + | DROP + | ELSE + | END + | ESCAPE + | ESCAPED + | EXCHANGE + | EXISTS + | EXPLAIN + | EXPORT + | EXTENDED + | EXTERNAL + | EXTRACT + | FALSE + | FETCH + | FILTER + | FIELDS + | FILEFORMAT + | FIRST + | FOLLOWING + | FOR + | FOREIGN + | FORMAT + | FORMATTED + | FROM + | FUNCTION + | FUNCTIONS + | GLOBAL + | GRANT + | GROUP + | GROUPING + | HAVING + | HOUR + | IF + | IGNORE + | IMPORT + | IN + | INDEX + | INDEXES + | INPATH + | INPUTFORMAT + | INSERT + | INTERVAL + | INTO + | IS + | ITEMS + | KEYS + | LAST + | LAZY + | LEADING + | LIKE + | LIMIT + | LINES + | LIST + | LOAD + | LOCAL + | LOCATION + | LOCK + | LOCKS + | LOGICAL + | MACRO + | MAP + | MATCHED + | MERGE + | MINUTE + | MONTH + | MSCK + | NAMESPACE + | NAMESPACES + | NO + | NOT + | NULL + | NULLS + | OF + | ONLY + | OPTION + | OPTIONS + | OR + | ORDER + | OUT + | OUTER + | OUTPUTFORMAT + | OVER + | OVERLAPS + | OVERLAY + | OVERWRITE + | PARTITION + | PARTITIONED + | PARTITIONS + | PERCENTLIT + | PIVOT + | PLACING + | POSITION + | PRECEDING + | PRIMARY + | PRINCIPALS + | PROPERTIES + | PURGE + | QUERY + | RANGE + | RECORDREADER + | RECORDWRITER + | RECOVER + | REDUCE + | REFERENCES + | REFRESH + | RENAME + | REPAIR + | REPLACE + | RESET + | RESPECT + | RESTRICT + | REVOKE + | RLIKE + | ROLE + | ROLES + | ROLLBACK + | ROLLUP + | ROW + | ROWS + | SCHEMA + | SECOND + | SELECT + | SEPARATED + | SERDE + | SERDEPROPERTIES + | SESSION_USER + | SET + | SETS + | SHOW + | SKEWED + | SOME + | SORT + | SORTED + | START + | STATISTICS + | STORED + | STRATIFY + | STRUCT + | SUBSTR + | SUBSTRING + | SYNC + | TABLE + | TABLES + | TABLESAMPLE + | TBLPROPERTIES + | TEMPORARY + | TERMINATED + | THEN + | TIME + | TO + | TOUCH + | TRAILING + | TRANSACTION + | TRANSACTIONS + | TRANSFORM + | TRIM + | TRUE + | TRUNCATE + | TRY_CAST + | TYPE + | UNARCHIVE + | UNBOUNDED + | UNCACHE + | UNIQUE + | UNKNOWN + | UNLOCK + | UNSET + | UPDATE + | USE + | USER + | VALUES + | VIEW + | VIEWS + | WHEN + | WHERE + | WINDOW + | WITH + | YEAR + | ZONE + | SYSTEM_VERSION + | VERSION + | SYSTEM_TIME + | TIMESTAMP +//--DEFAULT-NON-RESERVED-END + ; + +// NOTE: If you add a new token in the list below, you should update the list of keywords +// and reserved tag in `docs/sql-ref-ansi-compliance.md#sql-keywords`. + +//============================ +// Start of the keywords list +//============================ +//--SPARK-KEYWORD-LIST-START +ADD: 'ADD'; +AFTER: 'AFTER'; +ALL: 'ALL'; +ALTER: 'ALTER'; +ANALYZE: 'ANALYZE'; +AND: 'AND'; +ANTI: 'ANTI'; +ANY: 'ANY'; +ARCHIVE: 'ARCHIVE'; +ARRAY: 'ARRAY'; +AS: 'AS'; +ASC: 'ASC'; +AT: 'AT'; +AUTHORIZATION: 'AUTHORIZATION'; +BETWEEN: 'BETWEEN'; +BOTH: 'BOTH'; +BUCKET: 'BUCKET'; +BUCKETS: 'BUCKETS'; +BY: 'BY'; +CACHE: 'CACHE'; +CASCADE: 'CASCADE'; +CASE: 'CASE'; +CAST: 'CAST'; +CHANGE: 'CHANGE'; +CHECK: 'CHECK'; +CLEAR: 'CLEAR'; +CLUSTER: 'CLUSTER'; +CLUSTERED: 'CLUSTERED'; +CODEGEN: 'CODEGEN'; +COLLATE: 'COLLATE'; +COLLECTION: 'COLLECTION'; +COLUMN: 'COLUMN'; +COLUMNS: 'COLUMNS'; +COMMENT: 'COMMENT'; +COMMIT: 'COMMIT'; +COMPACT: 'COMPACT'; +COMPACTIONS: 'COMPACTIONS'; +COMPUTE: 'COMPUTE'; +CONCATENATE: 'CONCATENATE'; +CONSTRAINT: 'CONSTRAINT'; +COST: 'COST'; +CREATE: 'CREATE'; +CROSS: 'CROSS'; +CUBE: 'CUBE'; +CURRENT: 'CURRENT'; +CURRENT_DATE: 'CURRENT_DATE'; +CURRENT_TIME: 'CURRENT_TIME'; +CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; +CURRENT_USER: 'CURRENT_USER'; +DAY: 'DAY'; +DATA: 'DATA'; +DATABASE: 'DATABASE'; +DATABASES: 'DATABASES' | 'SCHEMAS'; +DBPROPERTIES: 'DBPROPERTIES'; +DEFINED: 'DEFINED'; +DELETE: 'DELETE'; +DELIMITED: 'DELIMITED'; +DESC: 'DESC'; +DESCRIBE: 'DESCRIBE'; +DFS: 'DFS'; +DIRECTORIES: 'DIRECTORIES'; +DIRECTORY: 'DIRECTORY'; +DISTINCT: 'DISTINCT'; +DISTRIBUTE: 'DISTRIBUTE'; +DIV: 'DIV'; +DROP: 'DROP'; +ELSE: 'ELSE'; +END: 'END'; +ESCAPE: 'ESCAPE'; +ESCAPED: 'ESCAPED'; +EXCEPT: 'EXCEPT'; +EXCHANGE: 'EXCHANGE'; +EXISTS: 'EXISTS'; +EXPLAIN: 'EXPLAIN'; +EXPORT: 'EXPORT'; +EXTENDED: 'EXTENDED'; +EXTERNAL: 'EXTERNAL'; +EXTRACT: 'EXTRACT'; +FALSE: 'FALSE'; +FETCH: 'FETCH'; +FIELDS: 'FIELDS'; +FILTER: 'FILTER'; +FILEFORMAT: 'FILEFORMAT'; +FIRST: 'FIRST'; +FOLLOWING: 'FOLLOWING'; +FOR: 'FOR'; +FOREIGN: 'FOREIGN'; +FORMAT: 'FORMAT'; +FORMATTED: 'FORMATTED'; +FROM: 'FROM'; +FULL: 'FULL'; +FUNCTION: 'FUNCTION'; +FUNCTIONS: 'FUNCTIONS'; +GLOBAL: 'GLOBAL'; +GRANT: 'GRANT'; +GROUP: 'GROUP'; +GROUPING: 'GROUPING'; +HAVING: 'HAVING'; +HOUR: 'HOUR'; +IF: 'IF'; +IGNORE: 'IGNORE'; +IMPORT: 'IMPORT'; +IN: 'IN'; +INDEX: 'INDEX'; +INDEXES: 'INDEXES'; +INNER: 'INNER'; +INPATH: 'INPATH'; +INPUTFORMAT: 'INPUTFORMAT'; +INSERT: 'INSERT'; +INTERSECT: 'INTERSECT'; +INTERVAL: 'INTERVAL'; +INTO: 'INTO'; +IS: 'IS'; +ITEMS: 'ITEMS'; +JOIN: 'JOIN'; +KEYS: 'KEYS'; +LAST: 'LAST'; +LATERAL: 'LATERAL'; +LAZY: 'LAZY'; +LEADING: 'LEADING'; +LEFT: 'LEFT'; +LIKE: 'LIKE'; +LIMIT: 'LIMIT'; +LINES: 'LINES'; +LIST: 'LIST'; +LOAD: 'LOAD'; +LOCAL: 'LOCAL'; +LOCATION: 'LOCATION'; +LOCK: 'LOCK'; +LOCKS: 'LOCKS'; +LOGICAL: 'LOGICAL'; +MACRO: 'MACRO'; +MAP: 'MAP'; +MATCHED: 'MATCHED'; +MERGE: 'MERGE'; +MINUTE: 'MINUTE'; +MONTH: 'MONTH'; +MSCK: 'MSCK'; +NAMESPACE: 'NAMESPACE'; +NAMESPACES: 'NAMESPACES'; +NATURAL: 'NATURAL'; +NO: 'NO'; +NOT: 'NOT' | '!'; +NULL: 'NULL'; +NULLS: 'NULLS'; +OF: 'OF'; +ON: 'ON'; +ONLY: 'ONLY'; +OPTION: 'OPTION'; +OPTIONS: 'OPTIONS'; +OR: 'OR'; +ORDER: 'ORDER'; +OUT: 'OUT'; +OUTER: 'OUTER'; +OUTPUTFORMAT: 'OUTPUTFORMAT'; +OVER: 'OVER'; +OVERLAPS: 'OVERLAPS'; +OVERLAY: 'OVERLAY'; +OVERWRITE: 'OVERWRITE'; +PARTITION: 'PARTITION'; +PARTITIONED: 'PARTITIONED'; +PARTITIONS: 'PARTITIONS'; +PERCENTLIT: 'PERCENT'; +PIVOT: 'PIVOT'; +PLACING: 'PLACING'; +POSITION: 'POSITION'; +PRECEDING: 'PRECEDING'; +PRIMARY: 'PRIMARY'; +PRINCIPALS: 'PRINCIPALS'; +PROPERTIES: 'PROPERTIES'; +PURGE: 'PURGE'; +QUERY: 'QUERY'; +RANGE: 'RANGE'; +RECORDREADER: 'RECORDREADER'; +RECORDWRITER: 'RECORDWRITER'; +RECOVER: 'RECOVER'; +REDUCE: 'REDUCE'; +REFERENCES: 'REFERENCES'; +REFRESH: 'REFRESH'; +RENAME: 'RENAME'; +REPAIR: 'REPAIR'; +REPLACE: 'REPLACE'; +RESET: 'RESET'; +RESPECT: 'RESPECT'; +RESTRICT: 'RESTRICT'; +REVOKE: 'REVOKE'; +RIGHT: 'RIGHT'; +RLIKE: 'RLIKE' | 'REGEXP'; +ROLE: 'ROLE'; +ROLES: 'ROLES'; +ROLLBACK: 'ROLLBACK'; +ROLLUP: 'ROLLUP'; +ROW: 'ROW'; +ROWS: 'ROWS'; +SECOND: 'SECOND'; +SCHEMA: 'SCHEMA'; +SELECT: 'SELECT'; +SEMI: 'SEMI'; +SEPARATED: 'SEPARATED'; +SERDE: 'SERDE'; +SERDEPROPERTIES: 'SERDEPROPERTIES'; +SESSION_USER: 'SESSION_USER'; +SET: 'SET'; +SETMINUS: 'MINUS'; +SETS: 'SETS'; +SHOW: 'SHOW'; +SKEWED: 'SKEWED'; +SOME: 'SOME'; +SORT: 'SORT'; +SORTED: 'SORTED'; +START: 'START'; +STATISTICS: 'STATISTICS'; +STORED: 'STORED'; +STRATIFY: 'STRATIFY'; +STRUCT: 'STRUCT'; +SUBSTR: 'SUBSTR'; +SUBSTRING: 'SUBSTRING'; +SYNC: 'SYNC'; +TABLE: 'TABLE'; +TABLES: 'TABLES'; +TABLESAMPLE: 'TABLESAMPLE'; +TBLPROPERTIES: 'TBLPROPERTIES'; +TEMPORARY: 'TEMPORARY' | 'TEMP'; +TERMINATED: 'TERMINATED'; +THEN: 'THEN'; +TIME: 'TIME'; +TO: 'TO'; +TOUCH: 'TOUCH'; +TRAILING: 'TRAILING'; +TRANSACTION: 'TRANSACTION'; +TRANSACTIONS: 'TRANSACTIONS'; +TRANSFORM: 'TRANSFORM'; +TRIM: 'TRIM'; +TRUE: 'TRUE'; +TRUNCATE: 'TRUNCATE'; +TRY_CAST: 'TRY_CAST'; +TYPE: 'TYPE'; +UNARCHIVE: 'UNARCHIVE'; +UNBOUNDED: 'UNBOUNDED'; +UNCACHE: 'UNCACHE'; +UNION: 'UNION'; +UNIQUE: 'UNIQUE'; +UNKNOWN: 'UNKNOWN'; +UNLOCK: 'UNLOCK'; +UNSET: 'UNSET'; +UPDATE: 'UPDATE'; +USE: 'USE'; +USER: 'USER'; +USING: 'USING'; +VALUES: 'VALUES'; +VIEW: 'VIEW'; +VIEWS: 'VIEWS'; +WHEN: 'WHEN'; +WHERE: 'WHERE'; +WINDOW: 'WINDOW'; +WITH: 'WITH'; +YEAR: 'YEAR'; +ZONE: 'ZONE'; + +SYSTEM_VERSION: 'SYSTEM_VERSION'; +VERSION: 'VERSION'; +SYSTEM_TIME: 'SYSTEM_TIME'; +TIMESTAMP: 'TIMESTAMP'; +//--SPARK-KEYWORD-LIST-END +//============================ +// End of the keywords list +//============================ + +EQ : '=' | '=='; +NSEQ: '<=>'; +NEQ : '<>'; +NEQJ: '!='; +LT : '<'; +LTE : '<=' | '!>'; +GT : '>'; +GTE : '>=' | '!<'; + +PLUS: '+'; +MINUS: '-'; +ASTERISK: '*'; +SLASH: '/'; +PERCENT: '%'; +TILDE: '~'; +AMPERSAND: '&'; +PIPE: '|'; +CONCAT_PIPE: '||'; +HAT: '^'; + +STRING + : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' + | '"' ( ~('"'|'\\') | ('\\' .) )* '"' + ; + +BIGINT_LITERAL + : DIGIT+ 'L' + ; + +SMALLINT_LITERAL + : DIGIT+ 'S' + ; + +TINYINT_LITERAL + : DIGIT+ 'Y' + ; + +INTEGER_VALUE + : DIGIT+ + ; + +EXPONENT_VALUE + : DIGIT+ EXPONENT + | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? + ; + +DECIMAL_VALUE + : DECIMAL_DIGITS {isValidDecimal()}? + ; + +FLOAT_LITERAL + : DIGIT+ EXPONENT? 'F' + | DECIMAL_DIGITS EXPONENT? 'F' {isValidDecimal()}? + ; + +DOUBLE_LITERAL + : DIGIT+ EXPONENT? 'D' + | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? + ; + +BIGDECIMAL_LITERAL + : DIGIT+ EXPONENT? 'BD' + | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? + ; + +IDENTIFIER + : (LETTER | DIGIT | '_')+ + ; + +BACKQUOTED_IDENTIFIER + : '`' ( ~'`' | '``' )* '`' + ; + +fragment DECIMAL_DIGITS + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + +fragment EXPONENT + : 'E' [+-]? DIGIT+ + ; + +fragment DIGIT + : [0-9] + ; + +fragment LETTER + : [A-Z] + ; + +SIMPLE_COMMENT + : '--' ('\\\n' | ~[\r\n])* '\r'? '\n'? -> channel(HIDDEN) + ; + +BRACKETED_COMMENT + : '/*' {!isHint()}? (BRACKETED_COMMENT|.)*? '*/' -> channel(HIDDEN) + ; + +WS + : [ \r\n\t]+ -> channel(HIDDEN) + ; + +// Catch-all for anything we can't recognize. +// We use this to be able to ignore and recover all the text +// when splitting statements with DelimiterLexer +UNRECOGNIZED + : . + ; diff --git a/hudi-spark-datasource/hudi-spark3/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 b/hudi-spark-datasource/hudi-spark3/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 new file mode 100644 index 0000000000000..585a7f1c2fb00 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlBase.g4 @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +grammar HoodieSqlBase; + +import SqlBase; + +singleStatement + : statement EOF + ; + +statement + : query #queryStatement + | ctes? dmlStatementNoWith #dmlStatement + | createTableHeader ('(' colTypeList ')')? tableProvider? + createTableClauses + (AS? query)? #createTable + | .*? #passThrough + ; diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala new file mode 100644 index 0000000000000..699623f8b58b0 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.adapter + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser + +/** + * The adapter for spark3.2. + */ +class Spark3_2Adapter extends Spark3Adapter { + /** + * if the logical plan is a TimeTravelRelation LogicalPlan. + */ + override def isRelationTimeTravel(plan: LogicalPlan): Boolean = { + plan.isInstanceOf[TimeTravelRelation] + } + + /** + * Get the member of the TimeTravelRelation LogicalPlan. + */ + override def getRelationTimeTravel(plan: LogicalPlan): Option[(LogicalPlan, Option[Expression], Option[String])] = { + plan match { + case timeTravel: TimeTravelRelation => + Some((timeTravel.table, timeTravel.timestamp, timeTravel.version)) + case _ => + None + } + } + + override def createExtendedSparkParser: Option[(SparkSession, ParserInterface) => ParserInterface] = { + Some( + (spark: SparkSession, delegate: ParserInterface) => new HoodieSpark3_2ExtendedSqlParser(spark, delegate) + ) + } +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala new file mode 100644 index 0000000000000..f243a7a86174f --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} + +case class TimeTravelRelation( + table: LogicalPlan, + timestamp: Option[Expression], + version: Option[String]) extends Command { + override def children: Seq[LogicalPlan] = Seq.empty + + override def output: Seq[Attribute] = Nil + + override lazy val resolved: Boolean = false + + def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = this +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlAstBuilder.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlAstBuilder.scala new file mode 100644 index 0000000000000..044e5984657ed --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlAstBuilder.scala @@ -0,0 +1,3342 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.parser + +import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} +import org.antlr.v4.runtime.{ParserRuleContext, Token} +import org.apache.hudi.spark.sql.parser.HoodieSqlBaseParser._ +import org.apache.hudi.spark.sql.parser.{HoodieSqlBaseBaseVisitor, HoodieSqlBaseParser} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} +import org.apache.spark.sql.catalyst.parser.ParserUtils.{EnhancedLogicalPlan, checkDuplicateClauses, checkDuplicateKeys, entry, escapedIdentifier, operationNotAllowed, source, string, stringWithoutUnescape, validate, withOrigin} +import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, IntervalUtils, truncatedString} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition +import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform, Expression => V2Expression} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.util.Utils.isTesting +import org.apache.spark.util.random.RandomSampler + +import java.util.Locale +import java.util.concurrent.TimeUnit +import javax.xml.bind.DatatypeConverter +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * The AstBuilder for HoodieSqlParser to parser the AST tree to Logical Plan. + * Here we only do the parser for the extended sql syntax. e.g MergeInto. For + * other sql syntax we use the delegate sql parser which is the SparkSqlParser. + */ +class HoodieSpark3_2ExtendedSqlAstBuilder(conf: SQLConf, delegate: ParserInterface) + extends HoodieSqlBaseBaseVisitor[AnyRef] with Logging { + + protected def typedVisit[T](ctx: ParseTree): T = { + ctx.accept(this).asInstanceOf[T] + } + + /** + * Override the default behavior for all visit methods. This will only return a non-null result + * when the context has only one child. This is done because there is no generic method to + * combine the results of the context children. In all other cases null is returned. + */ + override def visitChildren(node: RuleNode): AnyRef = { + if (node.getChildCount == 1) { + node.getChild(0).accept(this) + } else { + null + } + } + + /** + * Create an aliased table reference. This is typically used in FROM clauses. + */ + override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { + val tableId = visitMultipartIdentifier(ctx.multipartIdentifier()) + val relation = UnresolvedRelation(tableId) + val table = mayApplyAliasPlan( + ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) + table.optionalMap(ctx.sample)(withSample) + } + + private def withTimeTravel( + ctx: TemporalClauseContext, plan: LogicalPlan): LogicalPlan = withOrigin(ctx) { + val v = ctx.version + val version = if (ctx.INTEGER_VALUE != null) { + Some(v.getText) + } else { + Option(v).map(string) + } + + val timestamp = Option(ctx.timestamp).map(expression) + if (timestamp.exists(_.references.nonEmpty)) { + throw new ParseException( + "timestamp expression cannot refer to any columns", ctx.timestamp) + } + if (timestamp.exists(e => SubqueryExpression.hasSubquery(e))) { + throw new ParseException( + "timestamp expression cannot contain subqueries", ctx.timestamp) + } + + TimeTravelRelation(plan, timestamp, version) + } + + // ============== The following code is fork from org.apache.spark.sql.catalyst.parser.AstBuilder + override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) { + visit(ctx.statement).asInstanceOf[LogicalPlan] + } + + override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) { + visitNamedExpression(ctx.namedExpression) + } + + override def visitSingleTableIdentifier( + ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) { + visitTableIdentifier(ctx.tableIdentifier) + } + + override def visitSingleFunctionIdentifier( + ctx: SingleFunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) { + visitFunctionIdentifier(ctx.functionIdentifier) + } + + override def visitSingleMultipartIdentifier( + ctx: SingleMultipartIdentifierContext): Seq[String] = withOrigin(ctx) { + visitMultipartIdentifier(ctx.multipartIdentifier) + } + + override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) { + typedVisit[DataType](ctx.dataType) + } + + override def visitSingleTableSchema(ctx: SingleTableSchemaContext): StructType = { + val schema = StructType(visitColTypeList(ctx.colTypeList)) + withOrigin(ctx)(schema) + } + + /* ******************************************************************************************** + * Plan parsing + * ******************************************************************************************** */ + protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree) + + /** + * Create a top-level plan with Common Table Expressions. + */ + override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) { + val query = plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses) + + // Apply CTEs + query.optionalMap(ctx.ctes)(withCTE) + } + + override def visitDmlStatement(ctx: DmlStatementContext): AnyRef = withOrigin(ctx) { + val dmlStmt = plan(ctx.dmlStatementNoWith) + // Apply CTEs + dmlStmt.optionalMap(ctx.ctes)(withCTE) + } + + private def withCTE(ctx: CtesContext, plan: LogicalPlan): LogicalPlan = { + val ctes = ctx.namedQuery.asScala.map { nCtx => + val namedQuery = visitNamedQuery(nCtx) + (namedQuery.alias, namedQuery) + } + // Check for duplicate names. + val duplicates = ctes.groupBy(_._1).filter(_._2.size > 1).keys + if (duplicates.nonEmpty) { + throw new ParseException(s"CTE definition can't have duplicate names: ${duplicates.mkString("'", "', '", "'")}.", ctx) + } + UnresolvedWith(plan, ctes.toSeq) + } + + /** + * Create a logical query plan for a hive-style FROM statement body. + */ + private def withFromStatementBody( + ctx: FromStatementBodyContext, plan: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // two cases for transforms and selects + if (ctx.transformClause != null) { + withTransformQuerySpecification( + ctx, + ctx.transformClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + plan + ) + } else { + withSelectQuerySpecification( + ctx, + ctx.selectClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + plan + ) + } + } + + override def visitFromStatement(ctx: FromStatementContext): LogicalPlan = withOrigin(ctx) { + val from = visitFromClause(ctx.fromClause) + val selects = ctx.fromStatementBody.asScala.map { body => + withFromStatementBody(body, from). + // Add organization statements. + optionalMap(body.queryOrganization)(withQueryResultClauses) + } + // If there are multiple SELECT just UNION them together into one query. + if (selects.length == 1) { + selects.head + } else { + Union(selects.toSeq) + } + } + + /** + * Create a named logical plan. + * + * This is only used for Common Table Expressions. + */ + override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) { + val subQuery: LogicalPlan = plan(ctx.query).optionalMap(ctx.columnAliases)( + (columnAliases, plan) => + UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan) + ) + SubqueryAlias(ctx.name.getText, subQuery) + } + + /** + * Create a logical plan which allows for multiple inserts using one 'from' statement. These + * queries have the following SQL form: + * {{{ + * [WITH cte...]? + * FROM src + * [INSERT INTO tbl1 SELECT *]+ + * }}} + * For example: + * {{{ + * FROM db.tbl1 A + * INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5 + * INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12 + * }}} + * This (Hive) feature cannot be combined with set-operators. + */ + override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) { + val from = visitFromClause(ctx.fromClause) + + // Build the insert clauses. + val inserts = ctx.multiInsertQueryBody.asScala.map { body => + withInsertInto(body.insertInto, + withFromStatementBody(body.fromStatementBody, from). + optionalMap(body.fromStatementBody.queryOrganization)(withQueryResultClauses)) + } + + // If there are multiple INSERTS just UNION them together into one query. + if (inserts.length == 1) { + inserts.head + } else { + Union(inserts.toSeq) + } + } + + /** + * Create a logical plan for a regular (single-insert) query. + */ + override def visitSingleInsertQuery( + ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) { + withInsertInto( + ctx.insertInto(), + plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses)) + } + + /** + * Parameters used for writing query to a table: + * (UnresolvedRelation, tableColumnList, partitionKeys, ifPartitionNotExists). + */ + type InsertTableParams = (UnresolvedRelation, Seq[String], Map[String, Option[String]], Boolean) + + /** + * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). + */ + type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String]) + + /** + * Add an + * {{{ + * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? [identifierList] + * INSERT INTO [TABLE] tableIdentifier [partitionSpec] [identifierList] + * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] + * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] + * }}} + * operation to logical plan + */ + private def withInsertInto( + ctx: InsertIntoContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + ctx match { + case table: InsertIntoTableContext => + val (relation, cols, partition, ifPartitionNotExists) = visitInsertIntoTable(table) + InsertIntoStatement( + relation, + partition, + cols, + query, + overwrite = false, + ifPartitionNotExists) + case table: InsertOverwriteTableContext => + val (relation, cols, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table) + InsertIntoStatement( + relation, + partition, + cols, + query, + overwrite = true, + ifPartitionNotExists) + case dir: InsertOverwriteDirContext => + val (isLocal, storage, provider) = visitInsertOverwriteDir(dir) + InsertIntoDir(isLocal, storage, provider, query, overwrite = true) + case hiveDir: InsertOverwriteHiveDirContext => + val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir) + InsertIntoDir(isLocal, storage, provider, query, overwrite = true) + case _ => + throw new ParseException("Invalid InsertIntoContext", ctx) + } + } + + /** + * Add an INSERT INTO TABLE operation to the logical plan. + */ + override def visitInsertIntoTable( + ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) { + val cols = Option(ctx.identifierList()).map(visitIdentifierList).getOrElse(Nil) + val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) + + if (ctx.EXISTS != null) { + operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx) + } + + (createUnresolvedRelation(ctx.multipartIdentifier), cols, partitionKeys, false) + } + + /** + * Add an INSERT OVERWRITE TABLE operation to the logical plan. + */ + override def visitInsertOverwriteTable( + ctx: InsertOverwriteTableContext): InsertTableParams = withOrigin(ctx) { + assert(ctx.OVERWRITE() != null) + val cols = Option(ctx.identifierList()).map(visitIdentifierList).getOrElse(Nil) + val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) + + val dynamicPartitionKeys: Map[String, Option[String]] = partitionKeys.filter(_._2.isEmpty) + if (ctx.EXISTS != null && dynamicPartitionKeys.nonEmpty) { + operationNotAllowed("IF NOT EXISTS with dynamic partitions: " + + dynamicPartitionKeys.keys.mkString(", "), ctx) + } + + (createUnresolvedRelation(ctx.multipartIdentifier), cols, partitionKeys, ctx.EXISTS() != null) + } + + /** + * Write to a directory, returning a [[InsertIntoDir]] logical plan. + */ + override def visitInsertOverwriteDir( + ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { + throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) + } + + /** + * Write to a directory, returning a [[InsertIntoDir]] logical plan. + */ + override def visitInsertOverwriteHiveDir( + ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) { + throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) + } + + private def getTableAliasWithoutColumnAlias( + ctx: TableAliasContext, op: String): Option[String] = { + if (ctx == null) { + None + } else { + val ident = ctx.strictIdentifier() + if (ctx.identifierList() != null) { + throw new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList()) + } + if (ident != null) Some(ident.getText) else None + } + } + + override def visitDeleteFromTable( + ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { + val table = createUnresolvedRelation(ctx.multipartIdentifier()) + val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) + } else { + None + } + DeleteFromTable(aliasedTable, predicate) + } + + override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { + val table = createUnresolvedRelation(ctx.multipartIdentifier()) + val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) + val assignments = withAssignments(ctx.setClause().assignmentList()) + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) + } else { + None + } + + UpdateTable(aliasedTable, assignments, predicate) + } + + private def withAssignments(assignCtx: AssignmentListContext): Seq[Assignment] = + withOrigin(assignCtx) { + assignCtx.assignment().asScala.map { assign => + Assignment(UnresolvedAttribute(visitMultipartIdentifier(assign.key)), + expression(assign.value)) + }.toSeq + } + + override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) { + val targetTable = createUnresolvedRelation(ctx.target) + val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE") + val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable) + + val sourceTableOrQuery = if (ctx.source != null) { + createUnresolvedRelation(ctx.source) + } else if (ctx.sourceQuery != null) { + visitQuery(ctx.sourceQuery) + } else { + throw new ParseException("Empty source for merge: you should specify a source" + + " table/subquery in merge.", ctx.source) + } + val sourceTableAlias = getTableAliasWithoutColumnAlias(ctx.sourceAlias, "MERGE") + val aliasedSource = + sourceTableAlias.map(SubqueryAlias(_, sourceTableOrQuery)).getOrElse(sourceTableOrQuery) + + val mergeCondition = expression(ctx.mergeCondition) + + val matchedActions = ctx.matchedClause().asScala.map { + clause => { + if (clause.matchedAction().DELETE() != null) { + DeleteAction(Option(clause.matchedCond).map(expression)) + } else if (clause.matchedAction().UPDATE() != null) { + val condition = Option(clause.matchedCond).map(expression) + if (clause.matchedAction().ASTERISK() != null) { + UpdateStarAction(condition) + } else { + UpdateAction(condition, withAssignments(clause.matchedAction().assignmentList())) + } + } else { + // It should not be here. + throw new ParseException(s"Unrecognized matched action: ${clause.matchedAction().getText}", + clause.matchedAction()) + } + } + } + val notMatchedActions = ctx.notMatchedClause().asScala.map { + clause => { + if (clause.notMatchedAction().INSERT() != null) { + val condition = Option(clause.notMatchedCond).map(expression) + if (clause.notMatchedAction().ASTERISK() != null) { + InsertStarAction(condition) + } else { + val columns = clause.notMatchedAction().columns.multipartIdentifier() + .asScala.map(attr => UnresolvedAttribute(visitMultipartIdentifier(attr))) + val values = clause.notMatchedAction().expression().asScala.map(expression) + if (columns.size != values.size) { + throw new ParseException("The number of inserted values cannot match the fields.", + clause.notMatchedAction()) + } + InsertAction(condition, columns.zip(values).map(kv => Assignment(kv._1, kv._2)).toSeq) + } + } else { + // It should not be here. + throw new ParseException(s"Unrecognized not matched action: ${clause.notMatchedAction().getText}", + clause.notMatchedAction()) + } + } + } + if (matchedActions.isEmpty && notMatchedActions.isEmpty) { + throw new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx) + } + // children being empty means that the condition is not set + val matchedActionSize = matchedActions.length + if (matchedActionSize >= 2 && !matchedActions.init.forall(_.condition.nonEmpty)) { + throw new ParseException("When there are more than one MATCHED clauses in a MERGE " + + "statement, only the last MATCHED clause can omit the condition.", ctx) + } + val notMatchedActionSize = notMatchedActions.length + if (notMatchedActionSize >= 2 && !notMatchedActions.init.forall(_.condition.nonEmpty)) { + throw new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " + + "statement, only the last NOT MATCHED clause can omit the condition.", ctx) + } + + MergeIntoTable( + aliasedTarget, + aliasedSource, + mergeCondition, + matchedActions.toSeq, + notMatchedActions.toSeq) + } + + /** + * Create a partition specification map. + */ + override def visitPartitionSpec( + ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) { + val legacyNullAsString = + conf.getConf(SQLConf.LEGACY_PARSE_NULL_PARTITION_SPEC_AS_STRING_LITERAL) + val parts = ctx.partitionVal.asScala.map { pVal => + val name = pVal.identifier.getText + val value = Option(pVal.constant).map(v => visitStringConstant(v, legacyNullAsString)) + name -> value + } + // Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values + // in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for + // partition columns will be done in analyzer. + if (conf.caseSensitiveAnalysis) { + checkDuplicateKeys(parts.toSeq, ctx) + } else { + checkDuplicateKeys(parts.map(kv => kv._1.toLowerCase(Locale.ROOT) -> kv._2).toSeq, ctx) + } + parts.toMap + } + + /** + * Create a partition specification map without optional values. + */ + protected def visitNonOptionalPartitionSpec( + ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) { + visitPartitionSpec(ctx).map { + case (key, None) => throw new ParseException(s"Found an empty partition key '$key'.", ctx) + case (key, Some(value)) => key -> value + } + } + + /** + * Convert a constant of any type into a string. This is typically used in DDL commands, and its + * main purpose is to prevent slight differences due to back to back conversions i.e.: + * String -> Literal -> String. + */ + protected def visitStringConstant( + ctx: ConstantContext, + legacyNullAsString: Boolean): String = withOrigin(ctx) { + expression(ctx) match { + case Literal(null, _) if !legacyNullAsString => null + case l@Literal(null, _) => l.toString + case l: Literal => + // TODO For v2 commands, we will cast the string back to its actual value, + // which is a waste and can be improved in the future. + Cast(l, StringType, Some(conf.sessionLocalTimeZone)).eval().toString + case other => + throw new IllegalArgumentException(s"Only literals are allowed in the " + + s"partition spec, but got ${other.sql}") + } + } + + /** + * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These + * clauses determine the shape (ordering/partitioning/rows) of the query result. + */ + private def withQueryResultClauses( + ctx: QueryOrganizationContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + import ctx._ + + // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. + val withOrder = if ( + !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { + // ORDER BY ... + Sort(order.asScala.map(visitSortItem).toSeq, global = true, query) + } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { + // SORT BY ... + Sort(sort.asScala.map(visitSortItem).toSeq, global = false, query) + } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { + // DISTRIBUTE BY ... + withRepartitionByExpression(ctx, expressionList(distributeBy), query) + } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) { + // SORT BY ... DISTRIBUTE BY ... + Sort( + sort.asScala.map(visitSortItem).toSeq, + global = false, + withRepartitionByExpression(ctx, expressionList(distributeBy), query)) + } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) { + // CLUSTER BY ... + val expressions = expressionList(clusterBy) + Sort( + expressions.map(SortOrder(_, Ascending)), + global = false, + withRepartitionByExpression(ctx, expressions, query)) + } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) { + // [EMPTY] + query + } else { + throw new ParseException( + "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx) + } + + // WINDOWS + val withWindow = withOrder.optionalMap(windowClause)(withWindowClause) + + // LIMIT + // - LIMIT ALL is the same as omitting the LIMIT clause + withWindow.optional(limit) { + Limit(typedVisit(limit), withWindow) + } + } + + /** + * Create a clause for DISTRIBUTE BY. + */ + protected def withRepartitionByExpression( + ctx: QueryOrganizationContext, + expressions: Seq[Expression], + query: LogicalPlan): LogicalPlan = { + throw new ParseException("DISTRIBUTE BY is not supported", ctx) + } + + override def visitTransformQuerySpecification( + ctx: TransformQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { + val from = OneRowRelation().optional(ctx.fromClause) { + visitFromClause(ctx.fromClause) + } + withTransformQuerySpecification( + ctx, + ctx.transformClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + from + ) + } + + override def visitRegularQuerySpecification( + ctx: RegularQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { + val from = OneRowRelation().optional(ctx.fromClause) { + visitFromClause(ctx.fromClause) + } + withSelectQuerySpecification( + ctx, + ctx.selectClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + from + ) + } + + override def visitNamedExpressionSeq( + ctx: NamedExpressionSeqContext): Seq[Expression] = { + Option(ctx).toSeq + .flatMap(_.namedExpression.asScala) + .map(typedVisit[Expression]) + } + + override def visitExpressionSeq(ctx: ExpressionSeqContext): Seq[Expression] = { + Option(ctx).toSeq + .flatMap(_.expression.asScala) + .map(typedVisit[Expression]) + } + + /** + * Create a logical plan using a having clause. + */ + private def withHavingClause( + ctx: HavingClauseContext, plan: LogicalPlan): LogicalPlan = { + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(ctx.booleanExpression) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + UnresolvedHaving(predicate, plan) + } + + /** + * Create a logical plan using a where clause. + */ + private def withWhereClause(ctx: WhereClauseContext, plan: LogicalPlan): LogicalPlan = { + Filter(expression(ctx.booleanExpression), plan) + } + + /** + * Add a hive-style transform (SELECT TRANSFORM/MAP/REDUCE) query specification to a logical plan. + */ + private def withTransformQuerySpecification( + ctx: ParserRuleContext, + transformClause: TransformClauseContext, + lateralView: java.util.List[LateralViewContext], + whereClause: WhereClauseContext, + aggregationClause: AggregationClauseContext, + havingClause: HavingClauseContext, + windowClause: WindowClauseContext, + relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { + if (transformClause.setQuantifier != null) { + throw new ParseException("TRANSFORM does not support DISTINCT/ALL in inputs", transformClause.setQuantifier) + } + // Create the attributes. + val (attributes, schemaLess) = if (transformClause.colTypeList != null) { + // Typed return columns. + (createSchema(transformClause.colTypeList).toAttributes, false) + } else if (transformClause.identifierSeq != null) { + // Untyped return columns. + val attrs = visitIdentifierSeq(transformClause.identifierSeq).map { name => + AttributeReference(name, StringType, nullable = true)() + } + (attrs, false) + } else { + (Seq(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) + } + + val plan = visitCommonSelectQueryClausePlan( + relation, + visitExpressionSeq(transformClause.expressionSeq), + lateralView, + whereClause, + aggregationClause, + havingClause, + windowClause, + isDistinct = false) + + ScriptTransformation( + string(transformClause.script), + attributes, + plan, + withScriptIOSchema( + ctx, + transformClause.inRowFormat, + transformClause.recordWriter, + transformClause.outRowFormat, + transformClause.recordReader, + schemaLess + ) + ) + } + + /** + * Add a regular (SELECT) query specification to a logical plan. The query specification + * is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT), + * aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place. + * + * Note that query hints are ignored (both by the parser and the builder). + */ + private def withSelectQuerySpecification( + ctx: ParserRuleContext, + selectClause: SelectClauseContext, + lateralView: java.util.List[LateralViewContext], + whereClause: WhereClauseContext, + aggregationClause: AggregationClauseContext, + havingClause: HavingClauseContext, + windowClause: WindowClauseContext, + relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { + val isDistinct = selectClause.setQuantifier() != null && + selectClause.setQuantifier().DISTINCT() != null + + val plan = visitCommonSelectQueryClausePlan( + relation, + visitNamedExpressionSeq(selectClause.namedExpressionSeq), + lateralView, + whereClause, + aggregationClause, + havingClause, + windowClause, + isDistinct) + + // Hint + selectClause.hints.asScala.foldRight(plan)(withHints) + } + + def visitCommonSelectQueryClausePlan( + relation: LogicalPlan, + expressions: Seq[Expression], + lateralView: java.util.List[LateralViewContext], + whereClause: WhereClauseContext, + aggregationClause: AggregationClauseContext, + havingClause: HavingClauseContext, + windowClause: WindowClauseContext, + isDistinct: Boolean): LogicalPlan = { + // Add lateral views. + val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate) + + // Add where. + val withFilter = withLateralView.optionalMap(whereClause)(withWhereClause) + + // Add aggregation or a project. + val namedExpressions = expressions.map { + case e: NamedExpression => e + case e: Expression => UnresolvedAlias(e) + } + + def createProject() = if (namedExpressions.nonEmpty) { + Project(namedExpressions, withFilter) + } else { + withFilter + } + + val withProject = if (aggregationClause == null && havingClause != null) { + if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { + // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. + val predicate = expression(havingClause.booleanExpression) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, createProject()) + } else { + // According to SQL standard, HAVING without GROUP BY means global aggregate. + withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter)) + } + } else if (aggregationClause != null) { + val aggregate = withAggregationClause(aggregationClause, namedExpressions, withFilter) + aggregate.optionalMap(havingClause)(withHavingClause) + } else { + // When hitting this branch, `having` must be null. + createProject() + } + + // Distinct + val withDistinct = if (isDistinct) { + Distinct(withProject) + } else { + withProject + } + + // Window + val withWindow = withDistinct.optionalMap(windowClause)(withWindowClause) + + withWindow + } + + // Script Transform's input/output format. + type ScriptIOFormat = + (Seq[(String, String)], Option[String], Seq[(String, String)], Option[String]) + + protected def getRowFormatDelimited(ctx: RowFormatDelimitedContext): ScriptIOFormat = { + // TODO we should use the visitRowFormatDelimited function here. However HiveScriptIOSchema + // expects a seq of pairs in which the old parsers' token names are used as keys. + // Transforming the result of visitRowFormatDelimited would be quite a bit messier than + // retrieving the key value pairs ourselves. + val entries = entry("TOK_TABLEROWFORMATFIELD", ctx.fieldsTerminatedBy) ++ + entry("TOK_TABLEROWFORMATCOLLITEMS", ctx.collectionItemsTerminatedBy) ++ + entry("TOK_TABLEROWFORMATMAPKEYS", ctx.keysTerminatedBy) ++ + entry("TOK_TABLEROWFORMATNULL", ctx.nullDefinedAs) ++ + Option(ctx.linesSeparatedBy).toSeq.map { token => + val value = string(token) + validate( + value == "\n", + s"LINES TERMINATED BY only supports newline '\\n' right now: $value", + ctx) + "TOK_TABLEROWFORMATLINES" -> value + } + + (entries, None, Seq.empty, None) + } + + /** + * Create a [[ScriptInputOutputSchema]]. + */ + protected def withScriptIOSchema( + ctx: ParserRuleContext, + inRowFormat: RowFormatContext, + recordWriter: Token, + outRowFormat: RowFormatContext, + recordReader: Token, + schemaLess: Boolean): ScriptInputOutputSchema = { + + def format(fmt: RowFormatContext): ScriptIOFormat = fmt match { + case c: RowFormatDelimitedContext => + getRowFormatDelimited(c) + + case c: RowFormatSerdeContext => + throw new ParseException("TRANSFORM with serde is only supported in hive mode", ctx) + + // SPARK-32106: When there is no definition about format, we return empty result + // to use a built-in default Serde in SparkScriptTransformationExec. + case null => + (Nil, None, Seq.empty, None) + } + + val (inFormat, inSerdeClass, inSerdeProps, reader) = format(inRowFormat) + + val (outFormat, outSerdeClass, outSerdeProps, writer) = format(outRowFormat) + + ScriptInputOutputSchema( + inFormat, outFormat, + inSerdeClass, outSerdeClass, + inSerdeProps, outSerdeProps, + reader, writer, + schemaLess) + } + + /** + * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma + * separated) relations here, these get converted into a single plan by condition-less inner join. + */ + override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) { + val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) => + val right = plan(relation.relationPrimary) + val join = right.optionalMap(left) { (left, right) => + if (relation.LATERAL != null) { + if (!relation.relationPrimary.isInstanceOf[AliasedQueryContext]) { + throw new ParseException(s"LATERAL can only be used with subquery", relation.relationPrimary) + } + LateralJoin(left, LateralSubquery(right), Inner, None) + } else { + Join(left, right, Inner, None, JoinHint.NONE) + } + } + withJoinRelations(join, relation) + } + if (ctx.pivotClause() != null) { + if (!ctx.lateralView.isEmpty) { + throw new ParseException("LATERAL cannot be used together with PIVOT in FROM clause", ctx) + } + withPivot(ctx.pivotClause, from) + } else { + ctx.lateralView.asScala.foldLeft(from)(withGenerate) + } + } + + /** + * Connect two queries by a Set operator. + * + * Supported Set operators are: + * - UNION [ DISTINCT | ALL ] + * - EXCEPT [ DISTINCT | ALL ] + * - MINUS [ DISTINCT | ALL ] + * - INTERSECT [DISTINCT | ALL] + */ + override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) { + val left = plan(ctx.left) + val right = plan(ctx.right) + val all = Option(ctx.setQuantifier()).exists(_.ALL != null) + ctx.operator.getType match { + case HoodieSqlBaseParser.UNION if all => + Union(left, right) + case HoodieSqlBaseParser.UNION => + Distinct(Union(left, right)) + case HoodieSqlBaseParser.INTERSECT if all => + Intersect(left, right, isAll = true) + case HoodieSqlBaseParser.INTERSECT => + Intersect(left, right, isAll = false) + case HoodieSqlBaseParser.EXCEPT if all => + Except(left, right, isAll = true) + case HoodieSqlBaseParser.EXCEPT => + Except(left, right, isAll = false) + case HoodieSqlBaseParser.SETMINUS if all => + Except(left, right, isAll = true) + case HoodieSqlBaseParser.SETMINUS => + Except(left, right, isAll = false) + } + } + + /** + * Add a [[WithWindowDefinition]] operator to a logical plan. + */ + private def withWindowClause( + ctx: WindowClauseContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // Collect all window specifications defined in the WINDOW clause. + val baseWindowTuples = ctx.namedWindow.asScala.map { + wCtx => + (wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec)) + } + baseWindowTuples.groupBy(_._1).foreach { kv => + if (kv._2.size > 1) { + throw new ParseException(s"The definition of window '${kv._1}' is repetitive", ctx) + } + } + val baseWindowMap = baseWindowTuples.toMap + + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val windowMapView = baseWindowMap.mapValues { + case WindowSpecReference(name) => + baseWindowMap.get(name) match { + case Some(spec: WindowSpecDefinition) => + spec + case Some(ref) => + throw new ParseException(s"Window reference '$name' is not a window specification", ctx) + case None => + throw new ParseException(s"Cannot resolve window reference '$name'", ctx) + } + case spec: WindowSpecDefinition => spec + } + + // Note that mapValues creates a view instead of materialized map. We force materialization by + // mapping over identity. + WithWindowDefinition(windowMapView.map(identity).toMap, query) + } + + /** + * Add an [[Aggregate]] to a logical plan. + */ + private def withAggregationClause( + ctx: AggregationClauseContext, + selectExpressions: Seq[NamedExpression], + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + if (ctx.groupingExpressionsWithGroupingAnalytics.isEmpty) { + val groupByExpressions = expressionList(ctx.groupingExpressions) + if (ctx.GROUPING != null) { + // GROUP BY ... GROUPING SETS (...) + // `groupByExpressions` can be non-empty for Hive compatibility. It may add extra grouping + // expressions that do not exist in GROUPING SETS (...), and the value is always null. + // For example, `SELECT a, b, c FROM ... GROUP BY a, b, c GROUPING SETS (a, b)`, the output + // of column `c` is always null. + val groupingSets = + ctx.groupingSet.asScala.map(_.expression.asScala.map(e => expression(e)).toSeq) + Aggregate(Seq(GroupingSets(groupingSets.toSeq, groupByExpressions)), + selectExpressions, query) + } else { + // GROUP BY .... (WITH CUBE | WITH ROLLUP)? + val mappedGroupByExpressions = if (ctx.CUBE != null) { + Seq(Cube(groupByExpressions.map(Seq(_)))) + } else if (ctx.ROLLUP != null) { + Seq(Rollup(groupByExpressions.map(Seq(_)))) + } else { + groupByExpressions + } + Aggregate(mappedGroupByExpressions, selectExpressions, query) + } + } else { + val groupByExpressions = + ctx.groupingExpressionsWithGroupingAnalytics.asScala + .map(groupByExpr => { + val groupingAnalytics = groupByExpr.groupingAnalytics + if (groupingAnalytics != null) { + visitGroupingAnalytics(groupingAnalytics) + } else { + expression(groupByExpr.expression) + } + }) + Aggregate(groupByExpressions.toSeq, selectExpressions, query) + } + } + + override def visitGroupingAnalytics( + groupingAnalytics: GroupingAnalyticsContext): BaseGroupingSets = { + val groupingSets = groupingAnalytics.groupingSet.asScala + .map(_.expression.asScala.map(e => expression(e)).toSeq) + if (groupingAnalytics.CUBE != null) { + // CUBE(A, B, (A, B), ()) is not supported. + if (groupingSets.exists(_.isEmpty)) { + throw new ParseException(s"Empty set in CUBE grouping sets is not supported.", groupingAnalytics) + } + Cube(groupingSets.toSeq) + } else if (groupingAnalytics.ROLLUP != null) { + // ROLLUP(A, B, (A, B), ()) is not supported. + if (groupingSets.exists(_.isEmpty)) { + throw new ParseException(s"Empty set in ROLLUP grouping sets is not supported.", groupingAnalytics) + } + Rollup(groupingSets.toSeq) + } else { + assert(groupingAnalytics.GROUPING != null && groupingAnalytics.SETS != null) + val groupingSets = groupingAnalytics.groupingElement.asScala.flatMap { expr => + val groupingAnalytics = expr.groupingAnalytics() + if (groupingAnalytics != null) { + visitGroupingAnalytics(groupingAnalytics).selectedGroupByExprs + } else { + Seq(expr.groupingSet().expression().asScala.map(e => expression(e)).toSeq) + } + } + GroupingSets(groupingSets.toSeq) + } + } + + /** + * Add [[UnresolvedHint]]s to a logical plan. + */ + private def withHints( + ctx: HintContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + var plan = query + ctx.hintStatements.asScala.reverse.foreach { stmt => + plan = UnresolvedHint(stmt.hintName.getText, + stmt.parameters.asScala.map(expression).toSeq, plan) + } + plan + } + + /** + * Add a [[Pivot]] to a logical plan. + */ + private def withPivot( + ctx: PivotClauseContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + val aggregates = Option(ctx.aggregates).toSeq + .flatMap(_.namedExpression.asScala) + .map(typedVisit[Expression]) + val pivotColumn = if (ctx.pivotColumn.identifiers.size == 1) { + UnresolvedAttribute.quoted(ctx.pivotColumn.identifier.getText) + } else { + CreateStruct( + ctx.pivotColumn.identifiers.asScala.map( + identifier => UnresolvedAttribute.quoted(identifier.getText)).toSeq) + } + val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue) + Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query) + } + + /** + * Create a Pivot column value with or without an alias. + */ + override def visitPivotValue(ctx: PivotValueContext): Expression = withOrigin(ctx) { + val e = expression(ctx.expression) + if (ctx.identifier != null) { + Alias(e, ctx.identifier.getText)() + } else { + e + } + } + + /** + * Add a [[Generate]] (Lateral View) to a logical plan. + */ + private def withGenerate( + query: LogicalPlan, + ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) { + val expressions = expressionList(ctx.expression) + Generate( + UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions), + unrequiredChildIndex = Nil, + outer = ctx.OUTER != null, + // scalastyle:off caselocale + Some(ctx.tblName.getText.toLowerCase), + // scalastyle:on caselocale + ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.quoted).toSeq, + query) + } + + /** + * Create a single relation referenced in a FROM clause. This method is used when a part of the + * join condition is nested, for example: + * {{{ + * select * from t1 join (t2 cross join t3) on col1 = col2 + * }}} + */ + override def visitRelation(ctx: RelationContext): LogicalPlan = withOrigin(ctx) { + withJoinRelations(plan(ctx.relationPrimary), ctx) + } + + /** + * Join one more [[LogicalPlan]]s to the current logical plan. + */ + private def withJoinRelations(base: LogicalPlan, ctx: RelationContext): LogicalPlan = { + ctx.joinRelation.asScala.foldLeft(base) { (left, join) => + withOrigin(join) { + val baseJoinType = join.joinType match { + case null => Inner + case jt if jt.CROSS != null => Cross + case jt if jt.FULL != null => FullOuter + case jt if jt.SEMI != null => LeftSemi + case jt if jt.ANTI != null => LeftAnti + case jt if jt.LEFT != null => LeftOuter + case jt if jt.RIGHT != null => RightOuter + case _ => Inner + } + + if (join.LATERAL != null && !join.right.isInstanceOf[AliasedQueryContext]) { + throw new ParseException(s"LATERAL can only be used with subquery", join.right) + } + + // Resolve the join type and join condition + val (joinType, condition) = Option(join.joinCriteria) match { + case Some(c) if c.USING != null => + if (join.LATERAL != null) { + throw new ParseException("LATERAL join with USING join is not supported", ctx) + } + (UsingJoin(baseJoinType, visitIdentifierList(c.identifierList)), None) + case Some(c) if c.booleanExpression != null => + (baseJoinType, Option(expression(c.booleanExpression))) + case Some(c) => + throw new ParseException(s"Unimplemented joinCriteria: $c", ctx) + case None if join.NATURAL != null => + if (join.LATERAL != null) { + throw new ParseException("LATERAL join with NATURAL join is not supported", ctx) + } + if (baseJoinType == Cross) { + throw new ParseException("NATURAL CROSS JOIN is not supported", ctx) + } + (NaturalJoin(baseJoinType), None) + case None => + (baseJoinType, None) + } + if (join.LATERAL != null) { + if (!Seq(Inner, Cross, LeftOuter).contains(joinType)) { + throw new ParseException(s"Unsupported LATERAL join type ${joinType.toString}", ctx) + } + LateralJoin(left, LateralSubquery(plan(join.right)), joinType, condition) + } else { + Join(left, plan(join.right), joinType, condition, JoinHint.NONE) + } + } + } + } + + /** + * Add a [[Sample]] to a logical plan. + * + * This currently supports the following sampling methods: + * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows. + * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages + * are defined as a number between 0 and 100. + * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction. + */ + private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // Create a sampled plan if we need one. + def sample(fraction: Double): Sample = { + // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling + // function takes X PERCENT as the input and the range of X is [0, 100], we need to + // adjust the fraction. + val eps = RandomSampler.roundingEpsilon + validate(fraction >= 0.0 - eps && fraction <= 1.0 + eps, + s"Sampling fraction ($fraction) must be on interval [0, 1]", + ctx) + Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query) + } + + if (ctx.sampleMethod() == null) { + throw new ParseException("TABLESAMPLE does not accept empty inputs.", ctx) + } + + ctx.sampleMethod() match { + case ctx: SampleByRowsContext => + Limit(expression(ctx.expression), query) + + case ctx: SampleByPercentileContext => + val fraction = ctx.percentage.getText.toDouble + val sign = if (ctx.negativeSign == null) 1 else -1 + sample(sign * fraction / 100.0d) + + case ctx: SampleByBytesContext => + val bytesStr = ctx.bytes.getText + if (bytesStr.matches("[0-9]+[bBkKmMgG]")) { + throw new ParseException(s"TABLESAMPLE(byteLengthLiteral) is not supported", ctx) + } else { + throw new ParseException(s"$bytesStr is not a valid byte length literal, " + + "expected syntax: DIGIT+ ('B' | 'K' | 'M' | 'G')", ctx) + } + + case ctx: SampleByBucketContext if ctx.ON() != null => + if (ctx.identifier != null) { + throw new ParseException(s"TABLESAMPLE(BUCKET x OUT OF y ON colname) is not supported", ctx) + } else { + throw new ParseException(s"TABLESAMPLE(BUCKET x OUT OF y ON function) is not supported", ctx) + } + + case ctx: SampleByBucketContext => + sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble) + } + } + + /** + * Create a logical plan for a sub-query. + */ + override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) { + plan(ctx.query) + } + + /** + * Create an un-aliased table reference. This is typically used for top-level table references, + * for example: + * {{{ + * INSERT INTO db.tbl2 + * TABLE db.tbl1 + * }}} + */ + override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { + UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier)) + } + + /** + * Create a table-valued function call with arguments, e.g. range(1000) + */ + override def visitTableValuedFunction(ctx: TableValuedFunctionContext) + : LogicalPlan = withOrigin(ctx) { + val func = ctx.functionTable + val aliases = if (func.tableAlias.identifierList != null) { + visitIdentifierList(func.tableAlias.identifierList) + } else { + Seq.empty + } + val name = getFunctionIdentifier(func.functionName) + if (name.database.nonEmpty) { + operationNotAllowed(s"table valued function cannot specify database name: $name", ctx) + } + + val tvf = UnresolvedTableValuedFunction( + name, func.expression.asScala.map(expression).toSeq, aliases) + tvf.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) + } + + /** + * Create an inline table (a virtual table in Hive parlance). + */ + override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) { + // Get the backing expressions. + val rows = ctx.expression.asScala.map { e => + expression(e) match { + // inline table comes in two styles: + // style 1: values (1), (2), (3) -- multiple columns are supported + // style 2: values 1, 2, 3 -- only a single column is supported here + case struct: CreateNamedStruct => struct.valExprs // style 1 + case child => Seq(child) // style 2 + } + } + + val aliases = if (ctx.tableAlias.identifierList != null) { + visitIdentifierList(ctx.tableAlias.identifierList) + } else { + Seq.tabulate(rows.head.size)(i => s"col${i + 1}") + } + + val table = UnresolvedInlineTable(aliases, rows.toSeq) + table.optionalMap(ctx.tableAlias.strictIdentifier)(aliasPlan) + } + + /** + * Create an alias (SubqueryAlias) for a join relation. This is practically the same as + * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different + * hooks. We could add alias names for output columns, for example: + * {{{ + * SELECT a, b, c, d FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) + * }}} + */ + override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) { + val relation = plan(ctx.relation).optionalMap(ctx.sample)(withSample) + mayApplyAliasPlan(ctx.tableAlias, relation) + } + + /** + * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as + * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different + * hooks. We could add alias names for output columns, for example: + * {{{ + * SELECT col1, col2 FROM testData AS t(col1, col2) + * }}} + */ + override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) { + val relation = plan(ctx.query).optionalMap(ctx.sample)(withSample) + if (ctx.tableAlias.strictIdentifier == null) { + // For un-aliased subqueries, use a default alias name that is not likely to conflict with + // normal subquery names, so that parent operators can only access the columns in subquery by + // unqualified names. Users can still use this special qualifier to access columns if they + // know it, but that's not recommended. + SubqueryAlias("__auto_generated_subquery_name", relation) + } else { + mayApplyAliasPlan(ctx.tableAlias, relation) + } + } + + /** + * Create an alias ([[SubqueryAlias]]) for a [[LogicalPlan]]. + */ + private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = { + SubqueryAlias(alias.getText, plan) + } + + /** + * If aliases specified in a FROM clause, create a subquery alias ([[SubqueryAlias]]) and + * column aliases for a [[LogicalPlan]]. + */ + private def mayApplyAliasPlan(tableAlias: TableAliasContext, plan: LogicalPlan): LogicalPlan = { + if (tableAlias.strictIdentifier != null) { + val alias = tableAlias.strictIdentifier.getText + if (tableAlias.identifierList != null) { + val columnNames = visitIdentifierList(tableAlias.identifierList) + SubqueryAlias(alias, UnresolvedSubqueryColumnAliases(columnNames, plan)) + } else { + SubqueryAlias(alias, plan) + } + } else { + plan + } + } + + /** + * Create a Sequence of Strings for a parenthesis enclosed alias list. + */ + override def visitIdentifierList(ctx: IdentifierListContext): Seq[String] = withOrigin(ctx) { + visitIdentifierSeq(ctx.identifierSeq) + } + + /** + * Create a Sequence of Strings for an identifier list. + */ + override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { + ctx.ident.asScala.map(_.getText).toSeq + } + + /* ******************************************************************************************** + * Table Identifier parsing + * ******************************************************************************************** */ + + /** + * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern. + */ + override def visitTableIdentifier( + ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) { + TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText)) + } + + /** + * Create a [[FunctionIdentifier]] from a 'functionName' or 'databaseName'.'functionName' pattern. + */ + override def visitFunctionIdentifier( + ctx: FunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) { + FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText)) + } + + /** + * Create a multi-part identifier. + */ + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + ctx.parts.asScala.map(_.getText).toSeq + } + + /* ******************************************************************************************** + * Expression parsing + * ******************************************************************************************** */ + + /** + * Create an expression from the given context. This method just passes the context on to the + * visitor and only takes care of typing (We assume that the visitor returns an Expression here). + */ + protected def expression(ctx: ParserRuleContext): Expression = typedVisit(ctx) + + /** + * Create sequence of expressions from the given sequence of contexts. + */ + private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = { + trees.asScala.map(expression).toSeq + } + + /** + * Create a star (i.e. all) expression; this selects all elements (in the specified object). + * Both un-targeted (global) and targeted aliases are supported. + */ + override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) { + UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText).toSeq)) + } + + /** + * Create an aliased expression if an alias is specified. Both single and multi-aliases are + * supported. + */ + override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) { + val e = expression(ctx.expression) + if (ctx.name != null) { + Alias(e, ctx.name.getText)() + } else if (ctx.identifierList != null) { + MultiAlias(e, visitIdentifierList(ctx.identifierList)) + } else { + e + } + } + + /** + * Combine a number of boolean expressions into a balanced expression tree. These expressions are + * either combined by a logical [[And]] or a logical [[Or]]. + * + * A balanced binary tree is created because regular left recursive trees cause considerable + * performance degradations and can cause stack overflows. + */ + override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) { + val expressionType = ctx.operator.getType + val expressionCombiner = expressionType match { + case HoodieSqlBaseParser.AND => And.apply _ + case HoodieSqlBaseParser.OR => Or.apply _ + } + + // Collect all similar left hand contexts. + val contexts = ArrayBuffer(ctx.right) + var current = ctx.left + + def collectContexts: Boolean = current match { + case lbc: LogicalBinaryContext if lbc.operator.getType == expressionType => + contexts += lbc.right + current = lbc.left + true + case _ => + contexts += current + false + } + + while (collectContexts) { + // No body - all updates take place in the collectContexts. + } + + // Reverse the contexts to have them in the same sequence as in the SQL statement & turn them + // into expressions. + val expressions = contexts.reverseMap(expression) + + // Create a balanced tree. + def reduceToExpressionTree(low: Int, high: Int): Expression = high - low match { + case 0 => + expressions(low) + case 1 => + expressionCombiner(expressions(low), expressions(high)) + case x => + val mid = low + x / 2 + expressionCombiner( + reduceToExpressionTree(low, mid), + reduceToExpressionTree(mid + 1, high)) + } + + reduceToExpressionTree(0, expressions.size - 1) + } + + /** + * Invert a boolean expression. + */ + override def visitLogicalNot(ctx: LogicalNotContext): Expression = withOrigin(ctx) { + Not(expression(ctx.booleanExpression())) + } + + /** + * Create a filtering correlated sub-query (EXISTS). + */ + override def visitExists(ctx: ExistsContext): Expression = { + Exists(plan(ctx.query)) + } + + /** + * Create a comparison expression. This compares two expressions. The following comparison + * operators are supported: + * - Equal: '=' or '==' + * - Null-safe Equal: '<=>' + * - Not Equal: '<>' or '!=' + * - Less than: '<' + * - Less then or Equal: '<=' + * - Greater than: '>' + * - Greater then or Equal: '>=' + */ + override def visitComparison(ctx: ComparisonContext): Expression = withOrigin(ctx) { + val left = expression(ctx.left) + val right = expression(ctx.right) + val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode] + operator.getSymbol.getType match { + case HoodieSqlBaseParser.EQ => + EqualTo(left, right) + case HoodieSqlBaseParser.NSEQ => + EqualNullSafe(left, right) + case HoodieSqlBaseParser.NEQ | HoodieSqlBaseParser.NEQJ => + Not(EqualTo(left, right)) + case HoodieSqlBaseParser.LT => + LessThan(left, right) + case HoodieSqlBaseParser.LTE => + LessThanOrEqual(left, right) + case HoodieSqlBaseParser.GT => + GreaterThan(left, right) + case HoodieSqlBaseParser.GTE => + GreaterThanOrEqual(left, right) + } + } + + /** + * Create a predicated expression. A predicated expression is a normal expression with a + * predicate attached to it, for example: + * {{{ + * a + 1 IS NULL + * }}} + */ + override def visitPredicated(ctx: PredicatedContext): Expression = withOrigin(ctx) { + val e = expression(ctx.valueExpression) + if (ctx.predicate != null) { + withPredicate(e, ctx.predicate) + } else { + e + } + } + + /** + * Add a predicate to the given expression. Supported expressions are: + * - (NOT) BETWEEN + * - (NOT) IN + * - (NOT) LIKE (ANY | SOME | ALL) + * - (NOT) RLIKE + * - IS (NOT) NULL. + * - IS (NOT) (TRUE | FALSE | UNKNOWN) + * - IS (NOT) DISTINCT FROM + */ + private def withPredicate(e: Expression, ctx: PredicateContext): Expression = withOrigin(ctx) { + // Invert a predicate if it has a valid NOT clause. + def invertIfNotDefined(e: Expression): Expression = ctx.NOT match { + case null => e + case not => Not(e) + } + + def getValueExpressions(e: Expression): Seq[Expression] = e match { + case c: CreateNamedStruct => c.valExprs + case other => Seq(other) + } + + // Create the predicate. + ctx.kind.getType match { + case HoodieSqlBaseParser.BETWEEN => + // BETWEEN is translated to lower <= e && e <= upper + invertIfNotDefined(And( + GreaterThanOrEqual(e, expression(ctx.lower)), + LessThanOrEqual(e, expression(ctx.upper)))) + case HoodieSqlBaseParser.IN if ctx.query != null => + invertIfNotDefined(InSubquery(getValueExpressions(e), ListQuery(plan(ctx.query)))) + case HoodieSqlBaseParser.IN => + invertIfNotDefined(In(e, ctx.expression.asScala.map(expression).toSeq)) + case HoodieSqlBaseParser.LIKE => + Option(ctx.quantifier).map(_.getType) match { + case Some(HoodieSqlBaseParser.ANY) | Some(HoodieSqlBaseParser.SOME) => + validate(!ctx.expression.isEmpty, "Expected something between '(' and ')'.", ctx) + val expressions = expressionList(ctx.expression) + if (expressions.forall(_.foldable) && expressions.forall(_.dataType == StringType)) { + // If there are many pattern expressions, will throw StackOverflowError. + // So we use LikeAny or NotLikeAny instead. + val patterns = expressions.map(_.eval(EmptyRow).asInstanceOf[UTF8String]) + ctx.NOT match { + case null => LikeAny(e, patterns) + case _ => NotLikeAny(e, patterns) + } + } else { + ctx.expression.asScala.map(expression) + .map(p => invertIfNotDefined(new Like(e, p))).toSeq.reduceLeft(Or) + } + case Some(HoodieSqlBaseParser.ALL) => + validate(!ctx.expression.isEmpty, "Expected something between '(' and ')'.", ctx) + val expressions = expressionList(ctx.expression) + if (expressions.forall(_.foldable) && expressions.forall(_.dataType == StringType)) { + // If there are many pattern expressions, will throw StackOverflowError. + // So we use LikeAll or NotLikeAll instead. + val patterns = expressions.map(_.eval(EmptyRow).asInstanceOf[UTF8String]) + ctx.NOT match { + case null => LikeAll(e, patterns) + case _ => NotLikeAll(e, patterns) + } + } else { + ctx.expression.asScala.map(expression) + .map(p => invertIfNotDefined(new Like(e, p))).toSeq.reduceLeft(And) + } + case _ => + val escapeChar = Option(ctx.escapeChar).map(string).map { str => + if (str.length != 1) { + throw new ParseException("Invalid escape string. Escape string must contain only one character.", ctx) + } + str.charAt(0) + }.getOrElse('\\') + invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) + } + case HoodieSqlBaseParser.RLIKE => + invertIfNotDefined(RLike(e, expression(ctx.pattern))) + case HoodieSqlBaseParser.NULL if ctx.NOT != null => + IsNotNull(e) + case HoodieSqlBaseParser.NULL => + IsNull(e) + case HoodieSqlBaseParser.TRUE => ctx.NOT match { + case null => EqualNullSafe(e, Literal(true)) + case _ => Not(EqualNullSafe(e, Literal(true))) + } + case HoodieSqlBaseParser.FALSE => ctx.NOT match { + case null => EqualNullSafe(e, Literal(false)) + case _ => Not(EqualNullSafe(e, Literal(false))) + } + case HoodieSqlBaseParser.UNKNOWN => ctx.NOT match { + case null => IsUnknown(e) + case _ => IsNotUnknown(e) + } + case HoodieSqlBaseParser.DISTINCT if ctx.NOT != null => + EqualNullSafe(e, expression(ctx.right)) + case HoodieSqlBaseParser.DISTINCT => + Not(EqualNullSafe(e, expression(ctx.right))) + } + } + + /** + * Create a binary arithmetic expression. The following arithmetic operators are supported: + * - Multiplication: '*' + * - Division: '/' + * - Hive Long Division: 'DIV' + * - Modulo: '%' + * - Addition: '+' + * - Subtraction: '-' + * - Binary AND: '&' + * - Binary XOR + * - Binary OR: '|' + */ + override def visitArithmeticBinary(ctx: ArithmeticBinaryContext): Expression = withOrigin(ctx) { + val left = expression(ctx.left) + val right = expression(ctx.right) + ctx.operator.getType match { + case HoodieSqlBaseParser.ASTERISK => + Multiply(left, right) + case HoodieSqlBaseParser.SLASH => + Divide(left, right) + case HoodieSqlBaseParser.PERCENT => + Remainder(left, right) + case HoodieSqlBaseParser.DIV => + IntegralDivide(left, right) + case HoodieSqlBaseParser.PLUS => + Add(left, right) + case HoodieSqlBaseParser.MINUS => + Subtract(left, right) + case HoodieSqlBaseParser.CONCAT_PIPE => + Concat(left :: right :: Nil) + case HoodieSqlBaseParser.AMPERSAND => + BitwiseAnd(left, right) + case HoodieSqlBaseParser.HAT => + BitwiseXor(left, right) + case HoodieSqlBaseParser.PIPE => + BitwiseOr(left, right) + } + } + + /** + * Create a unary arithmetic expression. The following arithmetic operators are supported: + * - Plus: '+' + * - Minus: '-' + * - Bitwise Not: '~' + */ + override def visitArithmeticUnary(ctx: ArithmeticUnaryContext): Expression = withOrigin(ctx) { + val value = expression(ctx.valueExpression) + ctx.operator.getType match { + case HoodieSqlBaseParser.PLUS => + UnaryPositive(value) + case HoodieSqlBaseParser.MINUS => + UnaryMinus(value) + case HoodieSqlBaseParser.TILDE => + BitwiseNot(value) + } + } + + override def visitCurrentLike(ctx: CurrentLikeContext): Expression = withOrigin(ctx) { + if (conf.ansiEnabled) { + ctx.name.getType match { + case HoodieSqlBaseParser.CURRENT_DATE => + CurrentDate() + case HoodieSqlBaseParser.CURRENT_TIMESTAMP => + CurrentTimestamp() + case HoodieSqlBaseParser.CURRENT_USER => + CurrentUser() + } + } else { + // If the parser is not in ansi mode, we should return `UnresolvedAttribute`, in case there + // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP`. + UnresolvedAttribute.quoted(ctx.name.getText) + } + } + + /** + * Create a [[Cast]] expression. + */ + override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) { + val rawDataType = typedVisit[DataType](ctx.dataType()) + val dataType = CharVarcharUtils.replaceCharVarcharWithStringForCast(rawDataType) + val cast = ctx.name.getType match { + case HoodieSqlBaseParser.CAST => + Cast(expression(ctx.expression), dataType) + + case HoodieSqlBaseParser.TRY_CAST => + TryCast(expression(ctx.expression), dataType) + } + cast.setTagValue(Cast.USER_SPECIFIED_CAST, true) + cast + } + + /** + * Create a [[CreateStruct]] expression. + */ + override def visitStruct(ctx: StructContext): Expression = withOrigin(ctx) { + CreateStruct.create(ctx.argument.asScala.map(expression).toSeq) + } + + /** + * Create a [[First]] expression. + */ + override def visitFirst(ctx: FirstContext): Expression = withOrigin(ctx) { + val ignoreNullsExpr = ctx.IGNORE != null + First(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() + } + + /** + * Create a [[Last]] expression. + */ + override def visitLast(ctx: LastContext): Expression = withOrigin(ctx) { + val ignoreNullsExpr = ctx.IGNORE != null + Last(expression(ctx.expression), ignoreNullsExpr).toAggregateExpression() + } + + /** + * Create a Position expression. + */ + override def visitPosition(ctx: PositionContext): Expression = withOrigin(ctx) { + new StringLocate(expression(ctx.substr), expression(ctx.str)) + } + + /** + * Create a Extract expression. + */ + override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { + val arguments = Seq(Literal(ctx.field.getText), expression(ctx.source)) + UnresolvedFunction("extract", arguments, isDistinct = false) + } + + /** + * Create a Substring/Substr expression. + */ + override def visitSubstring(ctx: SubstringContext): Expression = withOrigin(ctx) { + if (ctx.len != null) { + Substring(expression(ctx.str), expression(ctx.pos), expression(ctx.len)) + } else { + new Substring(expression(ctx.str), expression(ctx.pos)) + } + } + + /** + * Create a Trim expression. + */ + override def visitTrim(ctx: TrimContext): Expression = withOrigin(ctx) { + val srcStr = expression(ctx.srcStr) + val trimStr = Option(ctx.trimStr).map(expression) + Option(ctx.trimOption).map(_.getType).getOrElse(HoodieSqlBaseParser.BOTH) match { + case HoodieSqlBaseParser.BOTH => + StringTrim(srcStr, trimStr) + case HoodieSqlBaseParser.LEADING => + StringTrimLeft(srcStr, trimStr) + case HoodieSqlBaseParser.TRAILING => + StringTrimRight(srcStr, trimStr) + case other => + throw new ParseException("Function trim doesn't support with " + + s"type $other. Please use BOTH, LEADING or TRAILING as trim type", ctx) + } + } + + /** + * Create a Overlay expression. + */ + override def visitOverlay(ctx: OverlayContext): Expression = withOrigin(ctx) { + val input = expression(ctx.input) + val replace = expression(ctx.replace) + val position = expression(ctx.position) + val lengthOpt = Option(ctx.length).map(expression) + lengthOpt match { + case Some(length) => Overlay(input, replace, position, length) + case None => new Overlay(input, replace, position) + } + } + + /** + * Create a (windowed) Function expression. + */ + override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { + // Create the function call. + val name = ctx.functionName.getText + val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) + // Call `toSeq`, otherwise `ctx.argument.asScala.map(expression)` is `Buffer` in Scala 2.13 + val arguments = ctx.argument.asScala.map(expression).toSeq match { + case Seq(UnresolvedStar(None)) + if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => + // Transform COUNT(*) into COUNT(1). + Seq(Literal(1)) + case expressions => + expressions + } + val filter = Option(ctx.where).map(expression(_)) + val ignoreNulls = + Option(ctx.nullsOption).map(_.getType == HoodieSqlBaseParser.IGNORE).getOrElse(false) + val function = UnresolvedFunction( + getFunctionMultiparts(ctx.functionName), arguments, isDistinct, filter, ignoreNulls) + + // Check if the function is evaluated in a windowed context. + ctx.windowSpec match { + case spec: WindowRefContext => + UnresolvedWindowExpression(function, visitWindowRef(spec)) + case spec: WindowDefContext => + WindowExpression(function, visitWindowDef(spec)) + case _ => function + } + } + + /** + * Create a function database (optional) and name pair. + */ + protected def visitFunctionName(ctx: QualifiedNameContext): FunctionIdentifier = { + visitFunctionName(ctx, ctx.identifier().asScala.map(_.getText).toSeq) + } + + /** + * Create a function database (optional) and name pair. + */ + private def visitFunctionName(ctx: ParserRuleContext, texts: Seq[String]): FunctionIdentifier = { + texts match { + case Seq(db, fn) => FunctionIdentifier(fn, Option(db)) + case Seq(fn) => FunctionIdentifier(fn, None) + case other => + throw new ParseException(s"Unsupported function name '${texts.mkString(".")}'", ctx) + } + } + + /** + * Get a function identifier consist by database (optional) and name. + */ + protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { + if (ctx.qualifiedName != null) { + visitFunctionName(ctx.qualifiedName) + } else { + FunctionIdentifier(ctx.getText, None) + } + } + + protected def getFunctionMultiparts(ctx: FunctionNameContext): Seq[String] = { + if (ctx.qualifiedName != null) { + ctx.qualifiedName().identifier().asScala.map(_.getText).toSeq + } else { + Seq(ctx.getText) + } + } + + /** + * Create an [[LambdaFunction]]. + */ + override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { + val arguments = ctx.identifier().asScala.map { name => + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) + } + val function = expression(ctx.expression).transformUp { + case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) + } + LambdaFunction(function, arguments.toSeq) + } + + /** + * Create a reference to a window frame, i.e. [[WindowSpecReference]]. + */ + override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) { + WindowSpecReference(ctx.name.getText) + } + + /** + * Create a window definition, i.e. [[WindowSpecDefinition]]. + */ + override def visitWindowDef(ctx: WindowDefContext): WindowSpecDefinition = withOrigin(ctx) { + // CLUSTER BY ... | PARTITION BY ... ORDER BY ... + val partition = ctx.partition.asScala.map(expression) + val order = ctx.sortItem.asScala.map(visitSortItem) + + // RANGE/ROWS BETWEEN ... + val frameSpecOption = Option(ctx.windowFrame).map { frame => + val frameType = frame.frameType.getType match { + case HoodieSqlBaseParser.RANGE => RangeFrame + case HoodieSqlBaseParser.ROWS => RowFrame + } + + SpecifiedWindowFrame( + frameType, + visitFrameBound(frame.start), + Option(frame.end).map(visitFrameBound).getOrElse(CurrentRow)) + } + + WindowSpecDefinition( + partition.toSeq, + order.toSeq, + frameSpecOption.getOrElse(UnspecifiedFrame)) + } + + /** + * Create or resolve a frame boundary expressions. + */ + override def visitFrameBound(ctx: FrameBoundContext): Expression = withOrigin(ctx) { + def value: Expression = { + val e = expression(ctx.expression) + validate(e.resolved && e.foldable, "Frame bound value must be a literal.", ctx) + e + } + + ctx.boundType.getType match { + case HoodieSqlBaseParser.PRECEDING if ctx.UNBOUNDED != null => + UnboundedPreceding + case HoodieSqlBaseParser.PRECEDING => + UnaryMinus(value) + case HoodieSqlBaseParser.CURRENT => + CurrentRow + case HoodieSqlBaseParser.FOLLOWING if ctx.UNBOUNDED != null => + UnboundedFollowing + case HoodieSqlBaseParser.FOLLOWING => + value + } + } + + /** + * Create a [[CreateStruct]] expression. + */ + override def visitRowConstructor(ctx: RowConstructorContext): Expression = withOrigin(ctx) { + CreateStruct(ctx.namedExpression().asScala.map(expression).toSeq) + } + + /** + * Create a [[ScalarSubquery]] expression. + */ + override def visitSubqueryExpression( + ctx: SubqueryExpressionContext): Expression = withOrigin(ctx) { + ScalarSubquery(plan(ctx.query)) + } + + /** + * Create a value based [[CaseWhen]] expression. This has the following SQL form: + * {{{ + * CASE [expression] + * WHEN [value] THEN [expression] + * ... + * ELSE [expression] + * END + * }}} + */ + override def visitSimpleCase(ctx: SimpleCaseContext): Expression = withOrigin(ctx) { + val e = expression(ctx.value) + val branches = ctx.whenClause.asScala.map { wCtx => + (EqualTo(e, expression(wCtx.condition)), expression(wCtx.result)) + } + CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) + } + + /** + * Create a condition based [[CaseWhen]] expression. This has the following SQL syntax: + * {{{ + * CASE + * WHEN [predicate] THEN [expression] + * ... + * ELSE [expression] + * END + * }}} + * + * @param ctx the parse tree + * */ + override def visitSearchedCase(ctx: SearchedCaseContext): Expression = withOrigin(ctx) { + val branches = ctx.whenClause.asScala.map { wCtx => + (expression(wCtx.condition), expression(wCtx.result)) + } + CaseWhen(branches.toSeq, Option(ctx.elseExpression).map(expression)) + } + + /** + * Currently only regex in expressions of SELECT statements are supported; in other + * places, e.g., where `(a)?+.+` = 2, regex are not meaningful. + */ + private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { + var parent = ctx.getParent + var rtn = false + while (parent != null) { + if (parent.isInstanceOf[NamedExpressionContext]) { + rtn = true + } + parent = parent.getParent + } + rtn + } + + /** + * Create a dereference expression. The return type depends on the type of the parent. + * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or + * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression, + * it can be [[UnresolvedExtractValue]]. + */ + override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { + val attr = ctx.fieldName.getText + expression(ctx.base) match { + case unresolved_attr@UnresolvedAttribute(nameParts) => + ctx.fieldName.getStart.getText match { + case escapedIdentifier(columnNameRegex) + if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) => + UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), + conf.caseSensitiveAnalysis) + case _ => + UnresolvedAttribute(nameParts :+ attr) + } + case e => + UnresolvedExtractValue(e, Literal(attr)) + } + } + + /** + * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex + * quoted in `` + */ + override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { + ctx.getStart.getText match { + case escapedIdentifier(columnNameRegex) + if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) => + UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) + case _ => + UnresolvedAttribute.quoted(ctx.getText) + } + + } + + /** + * Create an [[UnresolvedExtractValue]] expression, this is used for subscript access to an array. + */ + override def visitSubscript(ctx: SubscriptContext): Expression = withOrigin(ctx) { + UnresolvedExtractValue(expression(ctx.value), expression(ctx.index)) + } + + /** + * Create an expression for an expression between parentheses. This is need because the ANTLR + * visitor cannot automatically convert the nested context into an expression. + */ + override def visitParenthesizedExpression( + ctx: ParenthesizedExpressionContext): Expression = withOrigin(ctx) { + expression(ctx.expression) + } + + /** + * Create a [[SortOrder]] expression. + */ + override def visitSortItem(ctx: SortItemContext): SortOrder = withOrigin(ctx) { + val direction = if (ctx.DESC != null) { + Descending + } else { + Ascending + } + val nullOrdering = if (ctx.FIRST != null) { + NullsFirst + } else if (ctx.LAST != null) { + NullsLast + } else { + direction.defaultNullOrdering + } + SortOrder(expression(ctx.expression), direction, nullOrdering, Seq.empty) + } + + /** + * Create a typed Literal expression. A typed literal has the following SQL syntax: + * {{{ + * [TYPE] '[VALUE]' + * }}} + * Currently Date, Timestamp, Interval and Binary typed literals are supported. + */ + override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { + val value = string(ctx.STRING) + val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) + + def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { + f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { + throw new ParseException(s"Cannot parse the $valueType value: $value", ctx) + } + } + + def constructTimestampLTZLiteral(value: String): Literal = { + val zoneId = getZoneId(conf.sessionLocalTimeZone) + val specialTs = convertSpecialTimestamp(value, zoneId).map(Literal(_, TimestampType)) + specialTs.getOrElse(toLiteral(stringToTimestamp(_, zoneId), TimestampType)) + } + + try { + valueType match { + case "DATE" => + val zoneId = getZoneId(conf.sessionLocalTimeZone) + val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, DateType)) + specialDate.getOrElse(toLiteral(stringToDate, DateType)) + // SPARK-36227: Remove TimestampNTZ type support in Spark 3.2 with minimal code changes. + case "TIMESTAMP_NTZ" if isTesting => + convertSpecialTimestampNTZ(value, getZoneId(conf.sessionLocalTimeZone)) + .map(Literal(_, TimestampNTZType)) + .getOrElse(toLiteral(stringToTimestampWithoutTimeZone, TimestampNTZType)) + case "TIMESTAMP_LTZ" if isTesting => + constructTimestampLTZLiteral(value) + case "TIMESTAMP" => + SQLConf.get.timestampType match { + case TimestampNTZType => + convertSpecialTimestampNTZ(value, getZoneId(conf.sessionLocalTimeZone)) + .map(Literal(_, TimestampNTZType)) + .getOrElse { + val containsTimeZonePart = + DateTimeUtils.parseTimestampString(UTF8String.fromString(value))._2.isDefined + // If the input string contains time zone part, return a timestamp with local time + // zone literal. + if (containsTimeZonePart) { + constructTimestampLTZLiteral(value) + } else { + toLiteral(stringToTimestampWithoutTimeZone, TimestampNTZType) + } + } + + case TimestampType => + constructTimestampLTZLiteral(value) + } + + case "INTERVAL" => + val interval = try { + IntervalUtils.stringToInterval(UTF8String.fromString(value)) + } catch { + case e: IllegalArgumentException => + val ex = new ParseException(s"Cannot parse the INTERVAL value: $value", ctx) + ex.setStackTrace(e.getStackTrace) + throw ex + } + if (!conf.legacyIntervalEnabled) { + val units = value + .split("\\s") + .map(_.toLowerCase(Locale.ROOT).stripSuffix("s")) + .filter(s => s != "interval" && s.matches("[a-z]+")) + constructMultiUnitsIntervalLiteral(ctx, interval, units) + } else { + Literal(interval, CalendarIntervalType) + } + case "X" => + val padding = if (value.length % 2 != 0) "0" else "" + Literal(DatatypeConverter.parseHexBinary(padding + value)) + case other => + throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx) + } + } catch { + case e: IllegalArgumentException => + val message = Option(e.getMessage).getOrElse(s"Exception parsing $valueType") + throw new ParseException(message, ctx) + } + } + + /** + * Create a NULL literal expression. + */ + override def visitNullLiteral(ctx: NullLiteralContext): Literal = withOrigin(ctx) { + Literal(null) + } + + /** + * Create a Boolean literal expression. + */ + override def visitBooleanLiteral(ctx: BooleanLiteralContext): Literal = withOrigin(ctx) { + if (ctx.getText.toBoolean) { + Literal.TrueLiteral + } else { + Literal.FalseLiteral + } + } + + /** + * Create an integral literal expression. The code selects the most narrow integral type + * possible, either a BigDecimal, a Long or an Integer is returned. + */ + override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) { + BigDecimal(ctx.getText) match { + case v if v.isValidInt => + Literal(v.intValue) + case v if v.isValidLong => + Literal(v.longValue) + case v => Literal(v.underlying()) + } + } + + /** + * Create a decimal literal for a regular decimal number. + */ + override def visitDecimalLiteral(ctx: DecimalLiteralContext): Literal = withOrigin(ctx) { + Literal(BigDecimal(ctx.getText).underlying()) + } + + /** + * Create a decimal literal for a regular decimal number or a scientific decimal number. + */ + override def visitLegacyDecimalLiteral( + ctx: LegacyDecimalLiteralContext): Literal = withOrigin(ctx) { + Literal(BigDecimal(ctx.getText).underlying()) + } + + /** + * Create a double literal for number with an exponent, e.g. 1E-30 + */ + override def visitExponentLiteral(ctx: ExponentLiteralContext): Literal = { + numericLiteral(ctx, ctx.getText, /* exponent values don't have a suffix */ + Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) + } + + /** Create a numeric literal expression. */ + private def numericLiteral( + ctx: NumberContext, + rawStrippedQualifier: String, + minValue: BigDecimal, + maxValue: BigDecimal, + typeName: String)(converter: String => Any): Literal = withOrigin(ctx) { + try { + val rawBigDecimal = BigDecimal(rawStrippedQualifier) + if (rawBigDecimal < minValue || rawBigDecimal > maxValue) { + throw new ParseException(s"Numeric literal $rawStrippedQualifier does not " + + s"fit in range [$minValue, $maxValue] for type $typeName", ctx) + } + Literal(converter(rawStrippedQualifier)) + } catch { + case e: NumberFormatException => + throw new ParseException(e.getMessage, ctx) + } + } + + /** + * Create a Byte Literal expression. + */ + override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Byte.MinValue, Byte.MaxValue, ByteType.simpleString)(_.toByte) + } + + /** + * Create a Short Literal expression. + */ + override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Short.MinValue, Short.MaxValue, ShortType.simpleString)(_.toShort) + } + + /** + * Create a Long Literal expression. + */ + override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong) + } + + /** + * Create a Float Literal expression. + */ + override def visitFloatLiteral(ctx: FloatLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Float.MinValue, Float.MaxValue, FloatType.simpleString)(_.toFloat) + } + + /** + * Create a Double Literal expression. + */ + override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = { + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) + } + + /** + * Create a BigDecimal Literal expression. + */ + override def visitBigDecimalLiteral(ctx: BigDecimalLiteralContext): Literal = { + val raw = ctx.getText.substring(0, ctx.getText.length - 2) + try { + Literal(BigDecimal(raw).underlying()) + } catch { + case e: AnalysisException => + throw new ParseException(e.message, ctx) + } + } + + /** + * Create a String literal expression. + */ + override def visitStringLiteral(ctx: StringLiteralContext): Literal = withOrigin(ctx) { + Literal(createString(ctx)) + } + + /** + * Create a String from a string literal context. This supports multiple consecutive string + * literals, these are concatenated, for example this expression "'hello' 'world'" will be + * converted into "helloworld". + * + * Special characters can be escaped by using Hive/C-style escaping. + */ + private def createString(ctx: StringLiteralContext): String = { + if (conf.escapedStringLiterals) { + ctx.STRING().asScala.map(stringWithoutUnescape).mkString + } else { + ctx.STRING().asScala.map(string).mkString + } + } + + /** + * Create an [[UnresolvedRelation]] from a multi-part identifier context. + */ + private def createUnresolvedRelation( + ctx: MultipartIdentifierContext): UnresolvedRelation = withOrigin(ctx) { + UnresolvedRelation(visitMultipartIdentifier(ctx)) + } + + /** + * Construct an [[Literal]] from [[CalendarInterval]] and + * units represented as a [[Seq]] of [[String]]. + */ + private def constructMultiUnitsIntervalLiteral( + ctx: ParserRuleContext, + calendarInterval: CalendarInterval, + units: Seq[String]): Literal = { + var yearMonthFields = Set.empty[Byte] + var dayTimeFields = Set.empty[Byte] + for (unit <- units) { + if (YearMonthIntervalType.stringToField.contains(unit)) { + yearMonthFields += YearMonthIntervalType.stringToField(unit) + } else if (DayTimeIntervalType.stringToField.contains(unit)) { + dayTimeFields += DayTimeIntervalType.stringToField(unit) + } else if (unit == "week") { + dayTimeFields += DayTimeIntervalType.DAY + } else { + assert(unit == "millisecond" || unit == "microsecond") + dayTimeFields += DayTimeIntervalType.SECOND + } + } + if (yearMonthFields.nonEmpty) { + if (dayTimeFields.nonEmpty) { + val literalStr = source(ctx) + throw new ParseException(s"Cannot mix year-month and day-time fields: $literalStr", ctx) + } + Literal( + calendarInterval.months, + YearMonthIntervalType(yearMonthFields.min, yearMonthFields.max) + ) + } else { + Literal( + IntervalUtils.getDuration(calendarInterval, TimeUnit.MICROSECONDS), + DayTimeIntervalType(dayTimeFields.min, dayTimeFields.max)) + } + } + + /** + * Create a [[CalendarInterval]] or ANSI interval literal expression. + * Two syntaxes are supported: + * - multiple unit value pairs, for instance: interval 2 months 2 days. + * - from-to unit, for instance: interval '1-2' year to month. + */ + override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { + val calendarInterval = parseIntervalLiteral(ctx) + if (ctx.errorCapturingUnitToUnitInterval != null && !conf.legacyIntervalEnabled) { + // Check the `to` unit to distinguish year-month and day-time intervals because + // `CalendarInterval` doesn't have enough info. For instance, new CalendarInterval(0, 0, 0) + // can be derived from INTERVAL '0-0' YEAR TO MONTH as well as from + // INTERVAL '0 00:00:00' DAY TO SECOND. + val fromUnit = + ctx.errorCapturingUnitToUnitInterval.body.from.getText.toLowerCase(Locale.ROOT) + val toUnit = ctx.errorCapturingUnitToUnitInterval.body.to.getText.toLowerCase(Locale.ROOT) + if (toUnit == "month") { + assert(calendarInterval.days == 0 && calendarInterval.microseconds == 0) + val start = YearMonthIntervalType.stringToField(fromUnit) + Literal(calendarInterval.months, YearMonthIntervalType(start, YearMonthIntervalType.MONTH)) + } else { + assert(calendarInterval.months == 0) + val micros = IntervalUtils.getDuration(calendarInterval, TimeUnit.MICROSECONDS) + val start = DayTimeIntervalType.stringToField(fromUnit) + val end = DayTimeIntervalType.stringToField(toUnit) + Literal(micros, DayTimeIntervalType(start, end)) + } + } else if (ctx.errorCapturingMultiUnitsInterval != null && !conf.legacyIntervalEnabled) { + val units = + ctx.errorCapturingMultiUnitsInterval.body.unit.asScala.map( + _.getText.toLowerCase(Locale.ROOT).stripSuffix("s")).toSeq + constructMultiUnitsIntervalLiteral(ctx, calendarInterval, units) + } else { + Literal(calendarInterval, CalendarIntervalType) + } + } + + /** + * Create a [[CalendarInterval]] object + */ + protected def parseIntervalLiteral(ctx: IntervalContext): CalendarInterval = withOrigin(ctx) { + if (ctx.errorCapturingMultiUnitsInterval != null) { + val innerCtx = ctx.errorCapturingMultiUnitsInterval + if (innerCtx.unitToUnitInterval != null) { + throw new ParseException("Can only have a single from-to unit in the interval literal syntax", innerCtx.unitToUnitInterval) + } + visitMultiUnitsInterval(innerCtx.multiUnitsInterval) + } else if (ctx.errorCapturingUnitToUnitInterval != null) { + val innerCtx = ctx.errorCapturingUnitToUnitInterval + if (innerCtx.error1 != null || innerCtx.error2 != null) { + val errorCtx = if (innerCtx.error1 != null) innerCtx.error1 else innerCtx.error2 + throw new ParseException("Can only have a single from-to unit in the interval literal syntax", errorCtx) + } + visitUnitToUnitInterval(innerCtx.body) + } else { + throw new ParseException("at least one time unit should be given for interval literal", ctx) + } + } + + /** + * Creates a [[CalendarInterval]] with multiple unit value pairs, e.g. 1 YEAR 2 DAYS. + */ + override def visitMultiUnitsInterval(ctx: MultiUnitsIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val units = ctx.unit.asScala + val values = ctx.intervalValue().asScala + try { + assert(units.length == values.length) + val kvs = units.indices.map { i => + val u = units(i).getText + val v = if (values(i).STRING() != null) { + val value = string(values(i).STRING()) + // SPARK-32840: For invalid cases, e.g. INTERVAL '1 day 2' hour, + // INTERVAL 'interval 1' day, we need to check ahead before they are concatenated with + // units and become valid ones, e.g. '1 day 2 hour'. + // Ideally, we only ensure the value parts don't contain any units here. + if (value.exists(Character.isLetter)) { + throw new ParseException("Can only use numbers in the interval value part for" + + s" multiple unit value pairs interval form, but got invalid value: $value", ctx) + } + if (values(i).MINUS() == null) { + value + } else { + value.startsWith("-") match { + case true => value.replaceFirst("-", "") + case false => s"-$value" + } + } + } else { + values(i).getText + } + UTF8String.fromString(" " + v + " " + u) + } + IntervalUtils.stringToInterval(UTF8String.concat(kvs: _*)) + } catch { + case i: IllegalArgumentException => + val e = new ParseException(i.getMessage, ctx) + e.setStackTrace(i.getStackTrace) + throw e + } + } + } + + /** + * Creates a [[CalendarInterval]] with from-to unit, e.g. '2-1' YEAR TO MONTH. + */ + override def visitUnitToUnitInterval(ctx: UnitToUnitIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val value = Option(ctx.intervalValue.STRING).map(string).map { interval => + if (ctx.intervalValue().MINUS() == null) { + interval + } else { + interval.startsWith("-") match { + case true => interval.replaceFirst("-", "") + case false => s"-$interval" + } + } + }.getOrElse { + throw new ParseException("The value of from-to unit must be a string", ctx.intervalValue) + } + try { + val from = ctx.from.getText.toLowerCase(Locale.ROOT) + val to = ctx.to.getText.toLowerCase(Locale.ROOT) + (from, to) match { + case ("year", "month") => + IntervalUtils.fromYearMonthString(value) + case ("day", "hour") | ("day", "minute") | ("day", "second") | ("hour", "minute") | + ("hour", "second") | ("minute", "second") => + IntervalUtils.fromDayTimeString(value, + DayTimeIntervalType.stringToField(from), DayTimeIntervalType.stringToField(to)) + case _ => + throw new ParseException(s"Intervals FROM $from TO $to are not supported.", ctx) + } + } catch { + // Handle Exceptions thrown by CalendarInterval + case e: IllegalArgumentException => + val pe = new ParseException(e.getMessage, ctx) + pe.setStackTrace(e.getStackTrace) + throw pe + } + } + } + + /* ******************************************************************************************** + * DataType parsing + * ******************************************************************************************** */ + + /** + * Resolve/create a primitive type. + */ + override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) { + val dataType = ctx.identifier.getText.toLowerCase(Locale.ROOT) + (dataType, ctx.INTEGER_VALUE().asScala.toList) match { + case ("boolean", Nil) => BooleanType + case ("tinyint" | "byte", Nil) => ByteType + case ("smallint" | "short", Nil) => ShortType + case ("int" | "integer", Nil) => IntegerType + case ("bigint" | "long", Nil) => LongType + case ("float" | "real", Nil) => FloatType + case ("double", Nil) => DoubleType + case ("date", Nil) => DateType + case ("timestamp", Nil) => SQLConf.get.timestampType + // SPARK-36227: Remove TimestampNTZ type support in Spark 3.2 with minimal code changes. + case ("timestamp_ntz", Nil) if isTesting => TimestampNTZType + case ("timestamp_ltz", Nil) if isTesting => TimestampType + case ("string", Nil) => StringType + case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) + case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) + case ("binary", Nil) => BinaryType + case ("decimal" | "dec" | "numeric", Nil) => DecimalType.USER_DEFAULT + case ("decimal" | "dec" | "numeric", precision :: Nil) => + DecimalType(precision.getText.toInt, 0) + case ("decimal" | "dec" | "numeric", precision :: scale :: Nil) => + DecimalType(precision.getText.toInt, scale.getText.toInt) + case ("void", Nil) => NullType + case ("interval", Nil) => CalendarIntervalType + case (dt, params) => + val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt + throw new ParseException(s"DataType $dtStr is not supported.", ctx) + } + } + + override def visitYearMonthIntervalDataType(ctx: YearMonthIntervalDataTypeContext): DataType = { + val startStr = ctx.from.getText.toLowerCase(Locale.ROOT) + val start = YearMonthIntervalType.stringToField(startStr) + if (ctx.to != null) { + val endStr = ctx.to.getText.toLowerCase(Locale.ROOT) + val end = YearMonthIntervalType.stringToField(endStr) + if (end <= start) { + throw new ParseException(s"Intervals FROM $startStr TO $endStr are not supported.", ctx) + } + YearMonthIntervalType(start, end) + } else { + YearMonthIntervalType(start) + } + } + + override def visitDayTimeIntervalDataType(ctx: DayTimeIntervalDataTypeContext): DataType = { + val startStr = ctx.from.getText.toLowerCase(Locale.ROOT) + val start = DayTimeIntervalType.stringToField(startStr) + if (ctx.to != null) { + val endStr = ctx.to.getText.toLowerCase(Locale.ROOT) + val end = DayTimeIntervalType.stringToField(endStr) + if (end <= start) { + throw new ParseException(s"Intervals FROM $startStr TO $endStr are not supported.", ctx) + } + DayTimeIntervalType(start, end) + } else { + DayTimeIntervalType(start) + } + } + + /** + * Create a complex DataType. Arrays, Maps and Structures are supported. + */ + override def visitComplexDataType(ctx: ComplexDataTypeContext): DataType = withOrigin(ctx) { + ctx.complex.getType match { + case HoodieSqlBaseParser.ARRAY => + ArrayType(typedVisit(ctx.dataType(0))) + case HoodieSqlBaseParser.MAP => + MapType(typedVisit(ctx.dataType(0)), typedVisit(ctx.dataType(1))) + case HoodieSqlBaseParser.STRUCT => + StructType(Option(ctx.complexColTypeList).toSeq.flatMap(visitComplexColTypeList)) + } + } + + /** + * Create top level table schema. + */ + protected def createSchema(ctx: ColTypeListContext): StructType = { + StructType(Option(ctx).toSeq.flatMap(visitColTypeList)) + } + + /** + * Create a [[StructType]] from a number of column definitions. + */ + override def visitColTypeList(ctx: ColTypeListContext): Seq[StructField] = withOrigin(ctx) { + ctx.colType().asScala.map(visitColType).toSeq + } + + /** + * Create a top level [[StructField]] from a column definition. + */ + override def visitColType(ctx: ColTypeContext): StructField = withOrigin(ctx) { + import ctx._ + + val builder = new MetadataBuilder + // Add comment to metadata + Option(commentSpec()).map(visitCommentSpec).foreach { + builder.putString("comment", _) + } + + StructField( + name = colName.getText, + dataType = typedVisit[DataType](ctx.dataType), + nullable = NULL == null, + metadata = builder.build()) + } + + /** + * Create a [[StructType]] from a sequence of [[StructField]]s. + */ + protected def createStructType(ctx: ComplexColTypeListContext): StructType = { + StructType(Option(ctx).toSeq.flatMap(visitComplexColTypeList)) + } + + /** + * Create a [[StructType]] from a number of column definitions. + */ + override def visitComplexColTypeList( + ctx: ComplexColTypeListContext): Seq[StructField] = withOrigin(ctx) { + ctx.complexColType().asScala.map(visitComplexColType).toSeq + } + + /** + * Create a [[StructField]] from a column definition. + */ + override def visitComplexColType(ctx: ComplexColTypeContext): StructField = withOrigin(ctx) { + import ctx._ + val structField = StructField( + name = identifier.getText, + dataType = typedVisit(dataType()), + nullable = NULL == null) + Option(commentSpec).map(visitCommentSpec).map(structField.withComment).getOrElse(structField) + } + + /** + * Create a location string. + */ + override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { + string(ctx.STRING) + } + + /** + * Create an optional location string. + */ + protected def visitLocationSpecList(ctx: java.util.List[LocationSpecContext]): Option[String] = { + ctx.asScala.headOption.map(visitLocationSpec) + } + + /** + * Create a comment string. + */ + override def visitCommentSpec(ctx: CommentSpecContext): String = withOrigin(ctx) { + string(ctx.STRING) + } + + /** + * Create an optional comment string. + */ + protected def visitCommentSpecList(ctx: java.util.List[CommentSpecContext]): Option[String] = { + ctx.asScala.headOption.map(visitCommentSpec) + } + + /** + * Create a [[BucketSpec]]. + */ + override def visitBucketSpec(ctx: BucketSpecContext): BucketSpec = withOrigin(ctx) { + BucketSpec( + ctx.INTEGER_VALUE.getText.toInt, + visitIdentifierList(ctx.identifierList), + Option(ctx.orderedIdentifierList) + .toSeq + .flatMap(_.orderedIdentifier.asScala) + .map { orderedIdCtx => + Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => + if (dir.toLowerCase(Locale.ROOT) != "asc") { + operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) + } + } + + orderedIdCtx.ident.getText + }) + } + + /** + * Convert a table property list into a key-value map. + * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. + */ + override def visitTablePropertyList( + ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { + val properties = ctx.tableProperty.asScala.map { property => + val key = visitTablePropertyKey(property.key) + val value = visitTablePropertyValue(property.value) + key -> value + } + // Check for duplicate property names. + checkDuplicateKeys(properties.toSeq, ctx) + properties.toMap + } + + /** + * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. + */ + def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.collect { case (key, null) => key } + if (badKeys.nonEmpty) { + operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props + } + + /** + * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. + */ + def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v != null }.keys + if (badKeys.nonEmpty) { + operationNotAllowed( + s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props.keys.toSeq + } + + /** + * A table property key can either be String or a collection of dot separated elements. This + * function extracts the property key based on whether its a string literal or a table property + * identifier. + */ + override def visitTablePropertyKey(key: TablePropertyKeyContext): String = { + if (key.STRING != null) { + string(key.STRING) + } else { + key.getText + } + } + + /** + * A table property value can be String, Integer, Boolean or Decimal. This function extracts + * the property value based on whether its a string, integer, boolean or decimal literal. + */ + override def visitTablePropertyValue(value: TablePropertyValueContext): String = { + if (value == null) { + null + } else if (value.STRING != null) { + string(value.STRING) + } else if (value.booleanValue != null) { + value.getText.toLowerCase(Locale.ROOT) + } else { + value.getText + } + } + + /** + * Type to keep track of a table header: (identifier, isTemporary, ifNotExists, isExternal). + */ + type TableHeader = (Seq[String], Boolean, Boolean, Boolean) + + /** + * Type to keep track of table clauses: + * - partition transforms + * - partition columns + * - bucketSpec + * - properties + * - options + * - location + * - comment + * - serde + * + * Note: Partition transforms are based on existing table schema definition. It can be simple + * column names, or functions like `year(date_col)`. Partition columns are column names with data + * types like `i INT`, which should be appended to the existing table schema. + */ + type TableClauses = ( + Seq[Transform], Seq[StructField], Option[BucketSpec], Map[String, String], + Map[String, String], Option[String], Option[String], Option[SerdeInfo]) + + /** + * Validate a create table statement and return the [[TableIdentifier]]. + */ + override def visitCreateTableHeader( + ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { + val temporary = ctx.TEMPORARY != null + val ifNotExists = ctx.EXISTS != null + if (temporary && ifNotExists) { + operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + } + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq + (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) + } + + /** + * Validate a replace table statement and return the [[TableIdentifier]]. + */ + override def visitReplaceTableHeader( + ctx: ReplaceTableHeaderContext): TableHeader = withOrigin(ctx) { + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq + (multipartIdentifier, false, false, false) + } + + /** + * Parse a qualified name to a multipart name. + */ + override def visitQualifiedName(ctx: QualifiedNameContext): Seq[String] = withOrigin(ctx) { + ctx.identifier.asScala.map(_.getText).toSeq + } + + /** + * Parse a list of transforms or columns. + */ + override def visitPartitionFieldList( + ctx: PartitionFieldListContext): (Seq[Transform], Seq[StructField]) = withOrigin(ctx) { + val (transforms, columns) = ctx.fields.asScala.map { + case transform: PartitionTransformContext => + (Some(visitPartitionTransform(transform)), None) + case field: PartitionColumnContext => + (None, Some(visitColType(field.colType))) + }.unzip + + (transforms.flatten.toSeq, columns.flatten.toSeq) + } + + override def visitPartitionTransform( + ctx: PartitionTransformContext): Transform = withOrigin(ctx) { + def getFieldReference( + ctx: ApplyTransformContext, + arg: V2Expression): FieldReference = { + lazy val name: String = ctx.identifier.getText + arg match { + case ref: FieldReference => + ref + case nonRef => + throw new ParseException(s"Expected a column reference for transform $name: $nonRef.describe", ctx) + } + } + + def getSingleFieldReference( + ctx: ApplyTransformContext, + arguments: Seq[V2Expression]): FieldReference = { + lazy val name: String = ctx.identifier.getText + if (arguments.size > 1) { + throw new ParseException(s"Too many arguments for transform $name", ctx) + } else if (arguments.isEmpty) { + throw + + new ParseException(s"Not enough arguments for transform $name", ctx) + } else { + getFieldReference(ctx, arguments.head) + } + } + + ctx.transform match { + case identityCtx: IdentityTransformContext => + IdentityTransform(FieldReference(typedVisit[Seq[String]](identityCtx.qualifiedName))) + + case applyCtx: ApplyTransformContext => + val arguments = applyCtx.argument.asScala.map(visitTransformArgument).toSeq + + applyCtx.identifier.getText match { + case "bucket" => + val numBuckets: Int = arguments.head match { + case LiteralValue(shortValue, ShortType) => + shortValue.asInstanceOf[Short].toInt + case LiteralValue(intValue, IntegerType) => + intValue.asInstanceOf[Int] + case LiteralValue(longValue, LongType) => + longValue.asInstanceOf[Long].toInt + case lit => + throw new ParseException(s"Invalid number of buckets: ${lit.describe}", applyCtx) + } + + val fields = arguments.tail.map(arg => getFieldReference(applyCtx, arg)) + + BucketTransform(LiteralValue(numBuckets, IntegerType), fields) + + case "years" => + YearsTransform(getSingleFieldReference(applyCtx, arguments)) + + case "months" => + MonthsTransform(getSingleFieldReference(applyCtx, arguments)) + + case "days" => + DaysTransform(getSingleFieldReference(applyCtx, arguments)) + + case "hours" => + HoursTransform(getSingleFieldReference(applyCtx, arguments)) + + case name => + ApplyTransform(name, arguments) + } + } + } + + /** + * Parse an argument to a transform. An argument may be a field reference (qualified name) or + * a value literal. + */ + override def visitTransformArgument(ctx: TransformArgumentContext): V2Expression = { + withOrigin(ctx) { + val reference = Option(ctx.qualifiedName) + .map(typedVisit[Seq[String]]) + .map(FieldReference(_)) + val literal = Option(ctx.constant) + .map(typedVisit[Literal]) + .map(lit => LiteralValue(lit.value, lit.dataType)) + reference.orElse(literal) + .getOrElse(throw new ParseException("Invalid transform argument", ctx)) + } + } + + def cleanTableProperties( + ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { + import TableCatalog._ + val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) + properties.filter { + case (PROP_PROVIDER, _) if !legacyOn => + throw new ParseException(s"$PROP_PROVIDER is a reserved table property, please use the USING clause to specify it.", ctx) + case (PROP_PROVIDER, _) => false + case (PROP_LOCATION, _) if !legacyOn => + throw new ParseException(s"$PROP_LOCATION is a reserved table property, please use the LOCATION clause to specify it.", ctx) + case (PROP_LOCATION, _) => false + case (PROP_OWNER, _) if !legacyOn => + throw new ParseException(s"$PROP_OWNER is a reserved table property, it will be set to the current user.", ctx) + case (PROP_OWNER, _) => false + case _ => true + } + } + + def cleanTableOptions( + ctx: ParserRuleContext, + options: Map[String, String], + location: Option[String]): (Map[String, String], Option[String]) = { + var path = location + val filtered = cleanTableProperties(ctx, options).filter { + case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => + throw new ParseException(s"Duplicated table paths found: '${path.get}' and '$v'. LOCATION" + + s" and the case insensitive key 'path' in OPTIONS are all used to indicate the custom" + + s" table path, you can only specify one of them.", ctx) + case (k, v) if k.equalsIgnoreCase("path") => + path = Some(v) + false + case _ => true + } + (filtered, path) + } + + /** + * Create a [[SerdeInfo]] for creating tables. + * + * Format: STORED AS (name | INPUTFORMAT input_format OUTPUTFORMAT output_format) + */ + override def visitCreateFileFormat(ctx: CreateFileFormatContext): SerdeInfo = withOrigin(ctx) { + (ctx.fileFormat, ctx.storageHandler) match { + // Expected format: INPUTFORMAT input_format OUTPUTFORMAT output_format + case (c: TableFileFormatContext, null) => + SerdeInfo(formatClasses = Some(FormatClasses(string(c.inFmt), string(c.outFmt)))) + // Expected format: SEQUENCEFILE | TEXTFILE | RCFILE | ORC | PARQUET | AVRO + case (c: GenericFileFormatContext, null) => + SerdeInfo(storedAs = Some(c.identifier.getText)) + case (null, storageHandler) => + operationNotAllowed("STORED BY", ctx) + case _ => + throw new ParseException("Expected either STORED AS or STORED BY, not both", ctx) + } + } + + /** + * Create a [[SerdeInfo]] used for creating tables. + * + * Example format: + * {{{ + * SERDE serde_name [WITH SERDEPROPERTIES (k1=v1, k2=v2, ...)] + * }}} + * + * OR + * + * {{{ + * DELIMITED [FIELDS TERMINATED BY char [ESCAPED BY char]] + * [COLLECTION ITEMS TERMINATED BY char] + * [MAP KEYS TERMINATED BY char] + * [LINES TERMINATED BY char] + * [NULL DEFINED AS char] + * }}} + */ + def visitRowFormat(ctx: RowFormatContext): SerdeInfo = withOrigin(ctx) { + ctx match { + case serde: RowFormatSerdeContext => visitRowFormatSerde(serde) + case delimited: RowFormatDelimitedContext => visitRowFormatDelimited(delimited) + } + } + + /** + * Create SERDE row format name and properties pair. + */ + override def visitRowFormatSerde(ctx: RowFormatSerdeContext): SerdeInfo = withOrigin(ctx) { + import ctx._ + SerdeInfo( + serde = Some(string(name)), + serdeProperties = Option(tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) + } + + /** + * Create a delimited row format properties object. + */ + override def visitRowFormatDelimited( + ctx: RowFormatDelimitedContext): SerdeInfo = withOrigin(ctx) { + // Collect the entries if any. + def entry(key: String, value: Token): Seq[(String, String)] = { + Option(value).toSeq.map(x => key -> string(x)) + } + + // TODO we need proper support for the NULL format. + val entries = + entry("field.delim", ctx.fieldsTerminatedBy) ++ + entry("serialization.format", ctx.fieldsTerminatedBy) ++ + entry("escape.delim", ctx.escapedBy) ++ + // The following typo is inherited from Hive... + entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++ + entry("mapkey.delim", ctx.keysTerminatedBy) ++ + Option(ctx.linesSeparatedBy).toSeq.map { token => + val value = string(token) + validate( + value == "\n", + s"LINES TERMINATED BY only supports newline '\\n' right now: $value", + ctx) + "line.delim" -> value + } + SerdeInfo(serdeProperties = entries.toMap) + } + + /** + * Throw a [[ParseException]] if the user specified incompatible SerDes through ROW FORMAT + * and STORED AS. + * + * The following are allowed. Anything else is not: + * ROW FORMAT SERDE ... STORED AS [SEQUENCEFILE | RCFILE | TEXTFILE] + * ROW FORMAT DELIMITED ... STORED AS TEXTFILE + * ROW FORMAT ... STORED AS INPUTFORMAT ... OUTPUTFORMAT ... + */ + protected def validateRowFormatFileFormat( + rowFormatCtx: RowFormatContext, + createFileFormatCtx: CreateFileFormatContext, + parentCtx: ParserRuleContext): Unit = { + if (!(rowFormatCtx == null || createFileFormatCtx == null)) { + (rowFormatCtx, createFileFormatCtx.fileFormat) match { + case (_, ffTable: TableFileFormatContext) => // OK + case (rfSerde: RowFormatSerdeContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { + case ("sequencefile" | "textfile" | "rcfile") => // OK + case fmt => + operationNotAllowed( + s"ROW FORMAT SERDE is incompatible with format '$fmt', which also specifies a serde", + parentCtx) + } + case (rfDelimited: RowFormatDelimitedContext, ffGeneric: GenericFileFormatContext) => + ffGeneric.identifier.getText.toLowerCase(Locale.ROOT) match { + case "textfile" => // OK + case fmt => operationNotAllowed( + s"ROW FORMAT DELIMITED is only compatible with 'textfile', not '$fmt'", parentCtx) + } + case _ => + // should never happen + def str(ctx: ParserRuleContext): String = { + (0 until ctx.getChildCount).map { i => ctx.getChild(i).getText }.mkString(" ") + } + + operationNotAllowed( + s"Unexpected combination of ${str(rowFormatCtx)} and ${str(createFileFormatCtx)}", + parentCtx) + } + } + } + + protected def validateRowFormatFileFormat( + rowFormatCtx: Seq[RowFormatContext], + createFileFormatCtx: Seq[CreateFileFormatContext], + parentCtx: ParserRuleContext): Unit = { + if (rowFormatCtx.size == 1 && createFileFormatCtx.size == 1) { + validateRowFormatFileFormat(rowFormatCtx.head, createFileFormatCtx.head, parentCtx) + } + } + + override def visitCreateTableClauses(ctx: CreateTableClausesContext): TableClauses = { + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.createFileFormat, "STORED AS/BY", ctx) + checkDuplicateClauses(ctx.rowFormat, "ROW FORMAT", ctx) + checkDuplicateClauses(ctx.commentSpec(), "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + + if (ctx.skewSpec.size > 0) { + operationNotAllowed("CREATE TABLE ... SKEWED BY", ctx) + } + + val (partTransforms, partCols) = + Option(ctx.partitioning).map(visitPartitionFieldList).getOrElse((Nil, Nil)) + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val cleanedProperties = cleanTableProperties(ctx, properties) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val location = visitLocationSpecList(ctx.locationSpec()) + val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) + val comment = visitCommentSpecList(ctx.commentSpec()) + val serdeInfo = + getSerdeInfo(ctx.rowFormat.asScala.toSeq, ctx.createFileFormat.asScala.toSeq, ctx) + (partTransforms, partCols, bucketSpec, cleanedProperties, cleanedOptions, newLocation, comment, + serdeInfo) + } + + protected def getSerdeInfo( + rowFormatCtx: Seq[RowFormatContext], + createFileFormatCtx: Seq[CreateFileFormatContext], + ctx: ParserRuleContext): Option[SerdeInfo] = { + validateRowFormatFileFormat(rowFormatCtx, createFileFormatCtx, ctx) + val rowFormatSerdeInfo = rowFormatCtx.map(visitRowFormat) + val fileFormatSerdeInfo = createFileFormatCtx.map(visitCreateFileFormat) + (fileFormatSerdeInfo ++ rowFormatSerdeInfo).reduceLeftOption((l, r) => l.merge(r)) + } + + private def partitionExpressions( + partTransforms: Seq[Transform], + partCols: Seq[StructField], + ctx: ParserRuleContext): Seq[Transform] = { + if (partTransforms.nonEmpty) { + if (partCols.nonEmpty) { + val references = partTransforms.map(_.describe()).mkString(", ") + val columns = partCols + .map(field => s"${field.name} ${field.dataType.simpleString}") + .mkString(", ") + operationNotAllowed( + s"""PARTITION BY: Cannot mix partition expressions and partition columns: + |Expressions: $references + |Columns: $columns""".stripMargin, ctx) + + } + partTransforms + } else { + // columns were added to create the schema. convert to column references + partCols.map { column => + IdentityTransform(FieldReference(Seq(column.name))) + } + } + } + + /** + * Create a table, returning a [[CreateTableStatement]] logical plan. + * + * Expected format: + * {{{ + * CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name + * [USING table_provider] + * create_table_clauses + * [[AS] select_statement]; + * + * create_table_clauses (order insensitive): + * [PARTITIONED BY (partition_fields)] + * [OPTIONS table_property_list] + * [ROW FORMAT row_format] + * [STORED AS file_format] + * [CLUSTERED BY (col_name, col_name, ...) + * [SORTED BY (col_name [ASC|DESC], ...)] + * INTO num_buckets BUCKETS + * ] + * [LOCATION path] + * [COMMENT table_comment] + * [TBLPROPERTIES (property_name=property_value, ...)] + * + * partition_fields: + * col_name, transform(col_name), transform(constant, col_name), ... | + * col_name data_type [NOT NULL] [COMMENT col_comment], ... + * }}} + */ + override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = withOrigin(ctx) { + val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) + + val columns = Option(ctx.colTypeList()).map(visitColTypeList).getOrElse(Nil) + val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) + val (partTransforms, partCols, bucketSpec, properties, options, location, comment, serdeInfo) = + visitCreateTableClauses(ctx.createTableClauses()) + + if (provider.isDefined && serdeInfo.isDefined) { + operationNotAllowed(s"CREATE TABLE ... USING ... ${serdeInfo.get.describe}", ctx) + } + + if (temp) { + val asSelect = if (ctx.query == null) "" else " AS ..." + operationNotAllowed( + s"CREATE TEMPORARY TABLE ...$asSelect, use CREATE TEMPORARY VIEW instead", ctx) + } + + val partitioning = partitionExpressions(partTransforms, partCols, ctx) + + Option(ctx.query).map(plan) match { + case Some(_) if columns.nonEmpty => + operationNotAllowed( + "Schema may not be specified in a Create Table As Select (CTAS) statement", + ctx) + + case Some(_) if partCols.nonEmpty => + // non-reference partition columns are not allowed because schema can't be specified + operationNotAllowed( + "Partition column types may not be specified in Create Table As Select (CTAS)", + ctx) + + case Some(query) => + CreateTableAsSelectStatement( + table, query, partitioning, bucketSpec, properties, provider, options, location, comment, + writeOptions = Map.empty, serdeInfo, external = external, ifNotExists = ifNotExists) + + case _ => + // Note: table schema includes both the table columns list and the partition columns + // with data type. + val schema = StructType(columns ++ partCols) + CreateTableStatement(table, schema, partitioning, bucketSpec, properties, provider, + options, location, comment, serdeInfo, external = external, ifNotExists = ifNotExists) + } + } + + /** + * Parse new column info from ADD COLUMN into a QualifiedColType. + */ + override def visitQualifiedColTypeWithPosition( + ctx: QualifiedColTypeWithPositionContext): QualifiedColType = withOrigin(ctx) { + val name = typedVisit[Seq[String]](ctx.name) + QualifiedColType( + path = if (name.length > 1) Some(UnresolvedFieldName(name.init)) else None, + colName = name.last, + dataType = typedVisit[DataType](ctx.dataType), + nullable = ctx.NULL == null, + comment = Option(ctx.commentSpec()).map(visitCommentSpec), + position = Option(ctx.colPosition).map(pos => + UnresolvedFieldPosition(typedVisit[ColumnPosition](pos)))) + } +} + +/** + * A container for holding named common table expressions (CTEs) and a query plan. + * This operator will be removed during analysis and the relations will be substituted into child. + * + * @param child The final query of this CTE. + * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined + * Each CTE can see the base tables and the previously defined CTEs only. + */ +case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { + override def output: Seq[Attribute] = child.output + + override def simpleString(maxFields: Int): String = { + val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) + s"CTE $cteAliases" + } + + override def innerChildren: Seq[LogicalPlan] = cteRelations.map(_._2) + + def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = this +} diff --git a/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlParser.scala b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlParser.scala new file mode 100644 index 0000000000000..59ef8dfe0969b --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/parser/HoodieSpark3_2ExtendedSqlParser.scala @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parser + +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.atn.PredictionMode +import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} +import org.antlr.v4.runtime.tree.TerminalNodeImpl +import org.apache.hudi.spark.sql.parser.HoodieSqlBaseParser.{NonReservedContext, QuotedIdentifierContext} +import org.apache.hudi.spark.sql.parser.{HoodieSqlBaseBaseListener, HoodieSqlBaseLexer, HoodieSqlBaseParser} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.parser.{ParseErrorListener, ParseException, ParserInterface} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{AnalysisException, SparkSession} + +class HoodieSpark3_2ExtendedSqlParser(session: SparkSession, delegate: ParserInterface) + extends ParserInterface with Logging { + + private lazy val conf = session.sqlContext.conf + private lazy val builder = new HoodieSpark3_2ExtendedSqlAstBuilder(conf, delegate) + + override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser => + builder.visit(parser.singleStatement()) match { + case plan: LogicalPlan => plan + case _=> delegate.parsePlan(sqlText) + } + } + + override def parseExpression(sqlText: String): Expression = delegate.parseExpression(sqlText) + + override def parseTableIdentifier(sqlText: String): TableIdentifier = + delegate.parseTableIdentifier(sqlText) + + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = + delegate.parseFunctionIdentifier(sqlText) + + override def parseTableSchema(sqlText: String): StructType = delegate.parseTableSchema(sqlText) + + override def parseDataType(sqlText: String): DataType = delegate.parseDataType(sqlText) + + protected def parse[T](command: String)(toResult: HoodieSqlBaseParser => T): T = { + logDebug(s"Parsing command: $command") + + val lexer = new HoodieSqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) + lexer.removeErrorListeners() + lexer.addErrorListener(ParseErrorListener) + + val tokenStream = new CommonTokenStream(lexer) + val parser = new HoodieSqlBaseParser(tokenStream) + parser.addParseListener(PostProcessor) + parser.removeErrorListeners() + parser.addErrorListener(ParseErrorListener) +// parser.legacy_setops_precedence_enabled = conf.setOpsPrecedenceEnforced + parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled + parser.SQL_standard_keyword_behavior = conf.ansiEnabled + + try { + try { + // first, try parsing with potentially faster SLL mode + parser.getInterpreter.setPredictionMode(PredictionMode.SLL) + toResult(parser) + } + catch { + case e: ParseCancellationException => + // if we fail, parse with LL mode + tokenStream.seek(0) // rewind input stream + parser.reset() + + // Try Again. + parser.getInterpreter.setPredictionMode(PredictionMode.LL) + toResult(parser) + } + } + catch { + case e: ParseException if e.command.isDefined => + throw e + case e: ParseException => + throw e.withCommand(command) + case e: AnalysisException => + val position = Origin(e.line, e.startPosition) + throw new ParseException(Option(command), e.message, position, position) + } + } + + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } +} + +/** + * Fork from `org.apache.spark.sql.catalyst.parser.UpperCaseCharStream`. + */ +class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream { + override def consume(): Unit = wrapped.consume + override def getSourceName(): String = wrapped.getSourceName + override def index(): Int = wrapped.index + override def mark(): Int = wrapped.mark + override def release(marker: Int): Unit = wrapped.release(marker) + override def seek(where: Int): Unit = wrapped.seek(where) + override def size(): Int = wrapped.size + + override def getText(interval: Interval): String = { + // ANTLR 4.7's CodePointCharStream implementations have bugs when + // getText() is called with an empty stream, or intervals where + // the start > end. See + // https://github.com/antlr/antlr4/commit/ac9f7530 for one fix + // that is not yet in a released ANTLR artifact. + if (size() > 0 && (interval.b - interval.a >= 0)) { + wrapped.getText(interval) + } else { + "" + } + } + // scalastyle:off + override def LA(i: Int): Int = { + // scalastyle:on + val la = wrapped.LA(i) + if (la == 0 || la == IntStream.EOF) la + else Character.toUpperCase(la) + } +} + +/** + * Fork from `org.apache.spark.sql.catalyst.parser.PostProcessor`. + */ +case object PostProcessor extends HoodieSqlBaseBaseListener { + + /** Remove the back ticks from an Identifier. */ + override def exitQuotedIdentifier(ctx: QuotedIdentifierContext): Unit = { + replaceTokenByIdentifier(ctx, 1) { token => + // Remove the double back ticks in the string. + token.setText(token.getText.replace("``", "`")) + token + } + } + + /** Treat non-reserved keywords as Identifiers. */ + override def exitNonReserved(ctx: NonReservedContext): Unit = { + replaceTokenByIdentifier(ctx, 0)(identity) + } + + private def replaceTokenByIdentifier( + ctx: ParserRuleContext, + stripMargins: Int)( + f: CommonToken => CommonToken = identity): Unit = { + val parent = ctx.getParent + parent.removeLastChild() + val token = ctx.getChild(0).getPayload.asInstanceOf[Token] + val newToken = new CommonToken( + new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream), + HoodieSqlBaseParser.IDENTIFIER, + token.getChannel, + token.getStartIndex + stripMargins, + token.getStopIndex - stripMargins) + parent.addChild(new TerminalNodeImpl(f(newToken))) + } +} diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml index 7aaf39865e054..a19a603599d5a 100644 --- a/hudi-spark-datasource/pom.xml +++ b/hudi-spark-datasource/pom.xml @@ -33,11 +33,6 @@ hudi-spark-common hudi-spark - hudi-spark2 - hudi-spark3 - hudi-spark3.1.x - hudi-spark3-common - hudi-spark2-common diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 50991852b2c3b..8ee9daa836648 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -132,6 +132,9 @@ public class HiveSyncConfig implements Serializable { @Parameter(names = {"--spark-version"}, description = "The spark version", required = false) public String sparkVersion; + @Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive") + public boolean syncComment = false; + // enhance the similar function in child class public static HiveSyncConfig copy(HiveSyncConfig cfg) { HiveSyncConfig newConfig = new HiveSyncConfig(); @@ -159,6 +162,7 @@ public static HiveSyncConfig copy(HiveSyncConfig cfg) { newConfig.withOperationField = cfg.withOperationField; newConfig.isConditionalSync = cfg.isConditionalSync; newConfig.sparkVersion = cfg.sparkVersion; + newConfig.syncComment = cfg.syncComment; return newConfig; } @@ -193,6 +197,7 @@ public String toString() { + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold + ", withOperationField=" + withOperationField + ", isConditionalSync=" + isConditionalSync + + ", syncComment=" + syncComment + '}'; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 35200216ee9c0..952742b913330 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -19,9 +19,11 @@ package org.apache.hudi.hive; import com.beust.jcommander.JCommander; +import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; @@ -37,6 +39,7 @@ import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.hudi.sync.common.AbstractSyncTool; + import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.GroupType; @@ -261,6 +264,19 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea LOG.info("No Schema difference for " + tableName); } } + + if (cfg.syncComment) { + Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields(); + Map newComments = avroSchemaWithoutMetadataFields.getFields() + .stream().collect(Collectors.toMap(Schema.Field::name, field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); + boolean allEmpty = newComments.values().stream().allMatch(StringUtils::isNullOrEmpty); + if (!allEmpty) { + List hiveSchema = hoodieHiveClient.getTableCommentUsingMetastoreClient(tableName); + hoodieHiveClient.updateTableComments(tableName, hiveSchema, avroSchemaWithoutMetadataFields.getFields()); + } else { + LOG.info(String.format("No comment %s need to add", tableName)); + } + } return schemaChanged; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java index 1f8bcdf1c95b3..70a88a7aabbd1 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java @@ -19,21 +19,27 @@ package org.apache.hudi.hive; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.hive.ddl.DDLExecutor; import org.apache.hudi.hive.ddl.HMSDDLExecutor; import org.apache.hudi.hive.ddl.HiveQueryDDLExecutor; import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.hive.ddl.JDBCExecutor; +import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.AbstractSyncHoodieClient; +import org.apache.hudi.sync.common.HoodieSyncException; + +import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; @@ -46,7 +52,9 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.stream.Collectors; import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; @@ -343,4 +351,43 @@ public void updateLastCommitTimeSynced(String tableName) { } } } + + public Schema getAvroSchemaWithoutMetadataFields() { + try { + return new TableSchemaResolver(metaClient).getTableAvroSchemaWithoutMetadataFields(); + } catch (Exception e) { + throw new HoodieSyncException("Failed to read avro schema", e); + } + } + + public List getTableCommentUsingMetastoreClient(String tableName) { + try { + return client.getSchema(syncConfig.databaseName, tableName); + } catch (Exception e) { + throw new HoodieHiveSyncException("Failed to get table comments for : " + tableName, e); + } + } + + public void updateTableComments(String tableName, List oldSchema, List newSchema) { + Map newComments = newSchema.stream().collect(Collectors.toMap(field -> field.name().toLowerCase(Locale.ROOT), field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); + updateTableComments(tableName,oldSchema,newComments); + } + + public void updateTableComments(String tableName, List oldSchema, Map newComments) { + Map oldComments = oldSchema.stream().collect(Collectors.toMap(fieldSchema -> fieldSchema.getName().toLowerCase(Locale.ROOT), + fieldSchema -> StringUtils.isNullOrEmpty(fieldSchema.getComment()) ? "" : fieldSchema.getComment())); + Map types = oldSchema.stream().collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType)); + Map> alterComments = new HashMap<>(); + oldComments.forEach((name,comment) -> { + String newComment = newComments.getOrDefault(name,""); + if (!newComment.equals(comment)) { + alterComments.put(name,new ImmutablePair<>(types.get(name),newComment)); + } + }); + if (alterComments.size() > 0) { + ddlExecutor.updateTableComments(tableName, alterComments); + } else { + LOG.info(String.format("No comment difference of %s ",tableName)); + } + } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java index a7228584f611a..8cab505f1465b 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java @@ -18,6 +18,8 @@ package org.apache.hudi.hive.ddl; +import org.apache.hudi.common.util.collection.ImmutablePair; + import org.apache.parquet.schema.MessageType; import java.util.List; @@ -89,5 +91,13 @@ public void createTable(String tableName, MessageType storageSchema, String inpu */ public void dropPartitionsToTable(String tableName, List partitionsToDrop); + /** + * update table comments + * + * @param tableName + * @param newSchema + */ + public void updateTableComments(String tableName, Map> newSchema); + public void close(); } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java index c3c5226cd0a45..f2e9905350192 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java @@ -18,9 +18,9 @@ package org.apache.hudi.hive.ddl; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.StorageSchemes; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.hudi.hive.PartitionValueExtractor; @@ -43,6 +43,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; @@ -247,6 +248,27 @@ public void dropPartitionsToTable(String tableName, List partitionsToDro } } + @Override + public void updateTableComments(String tableName, Map> alterSchema) { + try { + Table table = client.getTable(syncConfig.databaseName, tableName); + StorageDescriptor sd = new StorageDescriptor(table.getSd()); + for (FieldSchema fieldSchema : sd.getCols()) { + if (alterSchema.containsKey(fieldSchema.getName())) { + String comment = alterSchema.get(fieldSchema.getName()).getRight(); + fieldSchema.setComment(comment); + } + } + table.setSd(sd); + EnvironmentContext environmentContext = new EnvironmentContext(); + client.alter_table_with_environmentContext(syncConfig.databaseName, tableName, table, environmentContext); + sd.clear(); + } catch (Exception e) { + LOG.error("Failed to update table comments for " + tableName, e); + throw new HoodieHiveSyncException("Failed to update table comments for " + tableName, e); + } + } + @Override public void close() { if (client != null) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java index a1cc7721053c5..d9b663ccb00ca 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.hudi.hive.PartitionValueExtractor; @@ -128,6 +129,24 @@ public void updatePartitionsToTable(String tableName, List changedPartit } } + @Override + public void updateTableComments(String tableName, Map> newSchema) { + for (Map.Entry> field : newSchema.entrySet()) { + String name = field.getKey(); + StringBuilder sql = new StringBuilder(); + String type = field.getValue().getLeft(); + String comment = field.getValue().getRight(); + comment = comment.replace("'",""); + sql.append("ALTER TABLE ").append(HIVE_ESCAPE_CHARACTER) + .append(config.databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") + .append(HIVE_ESCAPE_CHARACTER).append(tableName) + .append(HIVE_ESCAPE_CHARACTER) + .append(" CHANGE COLUMN `").append(name).append("` `").append(name) + .append("` ").append(type).append(" comment '").append(comment).append("' "); + runSQL(sql.toString()); + } + } + private List constructAddPartitions(String tableName, List partitions) { if (config.batchSyncNum <= 0) { throw new HoodieHiveSyncException("batch-sync-num for sync hive table must be greater than 0, pls check your parameter"); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 9a1012b649f47..9fc87fcb456b0 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.testutils.NetworkTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.ImmutablePair; import org.apache.hudi.hive.testutils.HiveTestUtil; import org.apache.hudi.hive.util.ConfigUtils; import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; @@ -33,6 +35,7 @@ import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.ql.Driver; @@ -52,7 +55,9 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.stream.Collectors; import static org.apache.hudi.hive.testutils.HiveTestUtil.ddlExecutor; import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; @@ -524,6 +529,77 @@ public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Excep "The last commit that was synced should be 101"); } + @ParameterizedTest + @MethodSource("syncMode") + public void testUpdateTableComments(String syncMode) throws Exception { + hiveSyncConfig.syncMode = syncMode; + String commitTime = "100"; + HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test.avsc"); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + + Map> alterCommentSchema = new HashMap<>(); + //generate commented schema field + Schema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, "/simple-test.avsc"); + Schema commentedSchema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, "/simple-test-doced.avsc"); + Map fieldsNameAndDoc = commentedSchema.getFields().stream().collect(Collectors.toMap(field -> field.name().toLowerCase(Locale.ROOT), + field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); + for (Field field : schema.getFields()) { + String name = field.name().toLowerCase(Locale.ROOT); + String comment = fieldsNameAndDoc.get(name); + if (fieldsNameAndDoc.containsKey(name) && !comment.equals(field.doc())) { + alterCommentSchema.put(name, new ImmutablePair<>(field.schema().getType().name(),comment)); + } + } + + ddlExecutor.updateTableComments(hiveSyncConfig.tableName,alterCommentSchema); + + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + int commentCnt = 0; + for (FieldSchema fieldSchema : fieldSchemas) { + if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + commentCnt++; + } + } + assertEquals(2, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); + } + + @ParameterizedTest + @MethodSource("syncMode") + public void testSyncWithCommentedSchema(String syncMode) throws Exception { + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.syncComment = false; + String commitTime = "100"; + HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test-doced.avsc"); + + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + int commentCnt = 0; + for (FieldSchema fieldSchema : fieldSchemas) { + if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + commentCnt++; + } + } + assertEquals(0, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); + + hiveSyncConfig.syncComment = true; + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); + commentCnt = 0; + for (FieldSchema fieldSchema : fieldSchemas) { + if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + commentCnt++; + } + } + assertEquals(2, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); + } + @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java index 4744fbb6b4e1c..1d3bb583a0861 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/RequestHandler.java @@ -123,18 +123,22 @@ private boolean isLocalViewBehind(Context ctx) { String timelineHashFromClient = ctx.queryParam(RemoteHoodieTableFileSystemView.TIMELINE_HASH, ""); HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline().filterCompletedAndCompactionInstants(); + String localLastKnownInstant = localTimeline.lastInstant().isPresent() ? localTimeline.lastInstant().get().getTimestamp() + : HoodieTimeline.INVALID_INSTANT_TS; if (LOG.isDebugEnabled()) { LOG.debug("Client [ LastTs=" + lastKnownInstantFromClient + ", TimelineHash=" + timelineHashFromClient + "], localTimeline=" + localTimeline.getInstants().collect(Collectors.toList())); } - if ((localTimeline.getInstants().count() == 0) + if ((!localTimeline.getInstants().findAny().isPresent()) && HoodieTimeline.INVALID_INSTANT_TS.equals(lastKnownInstantFromClient)) { return false; } String localTimelineHash = localTimeline.getTimelineHash(); - if (!localTimelineHash.equals(timelineHashFromClient)) { + // refresh if timeline hash mismatches and if local's last known instant < client's last known instant (if config is enabled) + if (!localTimelineHash.equals(timelineHashFromClient) + && (!timelineServiceConfig.refreshTimelineBasedOnLatestCommit || HoodieTimeline.compareTimestamps(localLastKnownInstant, HoodieTimeline.LESSER_THAN, lastKnownInstantFromClient))) { return true; } diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java index 7b8257705146e..40669f50e42d6 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java @@ -123,6 +123,9 @@ public static class Config implements Serializable { @Parameter(names = {"--marker-parallelism", "-mdp"}, description = "Parallelism to use for reading and deleting marker files") public int markerParallelism = 100; + @Parameter(names = {"--refreshTimelineBasedOnLatestCommit"}, description = "Refresh local timeline based on latest commit in addition to timeline hash value") + public boolean refreshTimelineBasedOnLatestCommit = true; + @Parameter(names = {"--help", "-h"}) public Boolean help = false; @@ -147,6 +150,7 @@ public static class Builder { private int markerBatchNumThreads = 20; private long markerBatchIntervalMs = 50L; private int markerParallelism = 100; + private boolean refreshTimelineBasedOnLatestCommit = false; public Builder() { } @@ -196,6 +200,11 @@ public Builder compress(boolean compress) { return this; } + public Builder refreshTimelineBasedOnLatestCommit(boolean refreshTimelineBasedOnLatestCommit) { + this.refreshTimelineBasedOnLatestCommit = refreshTimelineBasedOnLatestCommit; + return this; + } + public Builder enableMarkerRequests(boolean enableMarkerRequests) { this.enableMarkerRequests = enableMarkerRequests; return this; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java index f9b0e1a86d6af..be0044f275ca2 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataTableValidator.java @@ -709,7 +709,7 @@ public List getSortedBloomFilterList( .map(entry -> BloomFilterData.builder() .setPartitionPath(entry.getKey().getKey()) .setFilename(entry.getKey().getValue()) - .setBloomFilter(entry.getValue()) + .setBloomFilter(ByteBuffer.wrap(entry.getValue().serializeToString().getBytes())) .build()) .sorted() .collect(Collectors.toList()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 8690ff1cfb132..4dc0604ddbf21 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -18,7 +18,6 @@ package org.apache.hudi.utilities; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.AvroConversionUtils; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; @@ -44,6 +43,9 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; +import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException; +import org.apache.hudi.utilities.schema.ChainedSchemaPostProcessor; import org.apache.hudi.utilities.schema.DelegatingSchemaProvider; import org.apache.hudi.utilities.schema.RowBasedSchemaProvider; import org.apache.hudi.utilities.schema.SchemaPostProcessor; @@ -52,10 +54,13 @@ import org.apache.hudi.utilities.schema.SchemaProviderWithPostProcessor; import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; import org.apache.hudi.utilities.sources.Source; +import org.apache.hudi.utilities.sources.processor.ChainedJsonKafkaSourcePostProcessor; +import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; import org.apache.hudi.utilities.transform.ChainedTransformer; import org.apache.hudi.utilities.transform.Transformer; import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -122,8 +127,24 @@ public static Source createSource(String sourceClass, TypedProperties cfg, JavaS } } + public static JsonKafkaSourcePostProcessor createJsonKafkaSourcePostProcessor(String postProcessorClassNames, TypedProperties props) throws IOException { + if (StringUtils.isNullOrEmpty(postProcessorClassNames)) { + return null; + } + + try { + List processors = new ArrayList<>(); + for (String className : (postProcessorClassNames.split(","))) { + processors.add((JsonKafkaSourcePostProcessor) ReflectionUtils.loadClass(className, props)); + } + return new ChainedJsonKafkaSourcePostProcessor(processors, props); + } catch (Throwable e) { + throw new HoodieSourcePostProcessException("Could not load postProcessorClassNames class(es) " + postProcessorClassNames, e); + } + } + public static SchemaProvider createSchemaProvider(String schemaProviderClass, TypedProperties cfg, - JavaSparkContext jssc) throws IOException { + JavaSparkContext jssc) throws IOException { try { return StringUtils.isNullOrEmpty(schemaProviderClass) ? null : (SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc); @@ -133,10 +154,22 @@ public static SchemaProvider createSchemaProvider(String schemaProviderClass, Ty } public static SchemaPostProcessor createSchemaPostProcessor( - String schemaPostProcessorClass, TypedProperties cfg, JavaSparkContext jssc) { - return schemaPostProcessorClass == null - ? null - : (SchemaPostProcessor) ReflectionUtils.loadClass(schemaPostProcessorClass, cfg, jssc); + String schemaPostProcessorClassNames, TypedProperties cfg, JavaSparkContext jssc) { + + if (StringUtils.isNullOrEmpty(schemaPostProcessorClassNames)) { + return null; + } + + try { + List processors = new ArrayList<>(); + for (String className : (schemaPostProcessorClassNames.split(","))) { + processors.add((SchemaPostProcessor) ReflectionUtils.loadClass(className, cfg, jssc)); + } + return new ChainedSchemaPostProcessor(cfg, jssc, processors); + } catch (Throwable e) { + throw new HoodieSchemaPostProcessException("Could not load schemaPostProcessorClassNames class(es) " + schemaPostProcessorClassNames, e); + } + } public static Option createTransformer(List classNames) throws IOException { @@ -154,7 +187,7 @@ public static Option createTransformer(List classNames) thr public static InitialCheckPointProvider createInitialCheckpointProvider( String className, TypedProperties props) throws IOException { try { - return (InitialCheckPointProvider) ReflectionUtils.loadClass(className, new Class[] {TypedProperties.class}, props); + return (InitialCheckPointProvider) ReflectionUtils.loadClass(className, new Class[]{TypedProperties.class}, props); } catch (Throwable e) { throw new IOException("Could not load initial checkpoint provider class " + className, e); } @@ -411,21 +444,21 @@ public static SchemaProvider getOriginalSchemaProvider(SchemaProvider schemaProv } public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcessor(SchemaProvider provider, - TypedProperties cfg, JavaSparkContext jssc, List transformerClassNames) { + TypedProperties cfg, JavaSparkContext jssc, List transformerClassNames) { if (provider == null) { return null; } - if (provider instanceof SchemaProviderWithPostProcessor) { - return (SchemaProviderWithPostProcessor)provider; + if (provider instanceof SchemaProviderWithPostProcessor) { + return (SchemaProviderWithPostProcessor) provider; } String schemaPostProcessorClass = cfg.getString(Config.SCHEMA_POST_PROCESSOR_PROP, null); boolean enableSparkAvroPostProcessor = Boolean.parseBoolean(cfg.getString(SparkAvroPostProcessor.Config.SPARK_AVRO_POST_PROCESSOR_PROP_ENABLE, "true")); if (transformerClassNames != null && !transformerClassNames.isEmpty() - && enableSparkAvroPostProcessor && StringUtils.isNullOrEmpty(schemaPostProcessorClass)) { + && enableSparkAvroPostProcessor && StringUtils.isNullOrEmpty(schemaPostProcessorClass)) { schemaPostProcessorClass = SparkAvroPostProcessor.class.getName(); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 082a9b1d5e82d..2a319308fbfea 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -276,7 +276,7 @@ public void refreshTimeline() throws IOException { .setPartitionFields(partitionColumns) .setRecordKeyFields(props.getProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key())) .setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(), - Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))) + HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())) .setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), SimpleKeyGenerator.class.getName())) .setPreCombineField(cfg.sourceOrderingField) @@ -370,7 +370,7 @@ public Pair>> readFromSource( .setPartitionFields(partitionColumns) .setRecordKeyFields(props.getProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key())) .setPopulateMetaFields(props.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(), - Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))) + HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())) .setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), SimpleKeyGenerator.class.getName())) .initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSchemaPostProcessException.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSchemaPostProcessException.java new file mode 100644 index 0000000000000..dd765203d371c --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSchemaPostProcessException.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.exception; + +import org.apache.hudi.exception.HoodieException; + +/** + * Exception throws during schema post process. + */ +public class HoodieSchemaPostProcessException extends HoodieException { + public HoodieSchemaPostProcessException(String msg) { + super(msg); + } + + public HoodieSchemaPostProcessException(String message, Throwable t) { + super(message, t); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSourcePostProcessException.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSourcePostProcessException.java new file mode 100644 index 0000000000000..123d419e2dfda --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/exception/HoodieSourcePostProcessException.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.utilities.exception; + +import org.apache.hudi.exception.HoodieException; + +/** + * Exception throws during kafka source post process. + */ +public class HoodieSourcePostProcessException extends HoodieException { + + public HoodieSourcePostProcessException(String msg) { + super(msg); + } + + public HoodieSourcePostProcessException(String message, Throwable t) { + super(message, t); + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ChainedSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ChainedSchemaPostProcessor.java new file mode 100644 index 0000000000000..5a88d9fd92f3d --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/ChainedSchemaPostProcessor.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.schema; + +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.avro.Schema; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.List; + +/** + * A {@link SchemaPostProcessor} to chain other {@link SchemaPostProcessor}s and process sequentially. + */ +public class ChainedSchemaPostProcessor extends SchemaPostProcessor { + + private List processors; + + protected ChainedSchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + public ChainedSchemaPostProcessor(TypedProperties props, JavaSparkContext jssc, List processors) { + super(props, jssc); + this.processors = processors; + } + + @Override + public Schema processSchema(Schema schema) { + Schema targetSchema = schema; + for (SchemaPostProcessor processor : processors) { + targetSchema = processor.processSchema(targetSchema); + } + return targetSchema; + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java new file mode 100644 index 0000000000000..e58dc4e20611b --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DeleteSupportSchemaPostProcessor.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; + +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.ArrayList; +import java.util.List; + +/** + * An implementation of {@link SchemaPostProcessor} which will add a column named "_hoodie_is_deleted" to the end of + * a given schema. + */ +public class DeleteSupportSchemaPostProcessor extends SchemaPostProcessor { + + private static final Logger LOG = LogManager.getLogger(DeleteSupportSchemaPostProcessor.class); + + public DeleteSupportSchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + @Override + public Schema processSchema(Schema schema) { + + if (schema.getField(HoodieRecord.HOODIE_IS_DELETED) != null) { + LOG.warn(String.format("column %s already exists!", HoodieRecord.HOODIE_IS_DELETED)); + return schema; + } + + List sourceFields = schema.getFields(); + List targetFields = new ArrayList<>(sourceFields.size() + 1); + // copy existing columns + for (Schema.Field sourceField : sourceFields) { + targetFields.add(new Schema.Field(sourceField.name(), sourceField.schema(), sourceField.doc(), sourceField.defaultVal())); + } + // add _hoodie_is_deleted column + targetFields.add(new Schema.Field(HoodieRecord.HOODIE_IS_DELETED, Schema.create(Schema.Type.BOOLEAN), null, false)); + + return Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false, targetFields); + } + +} + + diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DropColumnSchemaPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DropColumnSchemaPostProcessor.java new file mode 100644 index 0000000000000..4a41b75589a3c --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/DropColumnSchemaPostProcessor.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException; + +import org.apache.avro.Schema; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A {@link SchemaPostProcessor} that support to delete column(s) from given schema. + *

+ * Multiple columns are separated by commas. + * For example: + *

+ * properties.put("hoodie.deltastreamer.schemaprovider.schema_post_processor.delete.columns", "column1,column2"). + */ +public class DropColumnSchemaPostProcessor extends SchemaPostProcessor { + + private static final Logger LOG = LogManager.getLogger(DropColumnSchemaPostProcessor.class); + + public DropColumnSchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + public static class Config { + public static final String DELETE_COLUMN_POST_PROCESSOR_COLUMN_PROP = + "hoodie.deltastreamer.schemaprovider.schema_post_processor.delete.columns"; + } + + @Override + public Schema processSchema(Schema schema) { + + String columnToDeleteStr = this.config.getString(Config.DELETE_COLUMN_POST_PROCESSOR_COLUMN_PROP); + + if (StringUtils.isNullOrEmpty(columnToDeleteStr)) { + LOG.warn(String.format("Param %s is null or empty, return original schema", Config.DELETE_COLUMN_POST_PROCESSOR_COLUMN_PROP)); + } + + // convert field to lowerCase for compare purpose + Set columnsToDelete = Arrays.stream(columnToDeleteStr.split(",")) + .map(filed -> filed.toLowerCase(Locale.ROOT)) + .collect(Collectors.toSet()); + + List sourceFields = schema.getFields(); + List targetFields = new LinkedList<>(); + + for (Schema.Field sourceField : sourceFields) { + if (!columnsToDelete.contains(sourceField.name().toLowerCase(Locale.ROOT))) { + targetFields.add(new Schema.Field(sourceField.name(), sourceField.schema(), sourceField.doc(), sourceField.defaultVal())); + } + } + + if (targetFields.isEmpty()) { + throw new HoodieSchemaPostProcessException("Target schema is empty, you can not remove all columns!"); + } + + return Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), false, targetFields); + } + +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java index 34549885cf8c9..3322b7b010385 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SparkAvroPostProcessor.java @@ -41,7 +41,7 @@ public SparkAvroPostProcessor(TypedProperties props, JavaSparkContext jssc) { @Override public Schema processSchema(Schema schema) { return schema != null ? AvroConversionUtils.convertStructTypeToAvroSchema( - AvroConversionUtils.convertAvroSchemaToStructType(schema), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME, + AvroConversionUtils.convertAvroSchemaToStructType(schema), RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME, RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE) : null; } } \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java index d6152a177f7fd..0b06d986bbf1a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/JsonKafkaSource.java @@ -21,11 +21,14 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics; +import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; import org.apache.hudi.utilities.exception.HoodieSourceTimeoutException; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen; import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; +import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.log4j.LogManager; @@ -37,6 +40,8 @@ import org.apache.spark.streaming.kafka010.LocationStrategies; import org.apache.spark.streaming.kafka010.OffsetRange; +import java.io.IOException; + /** * Read json kafka data. */ @@ -74,12 +79,30 @@ protected InputBatch> fetchNewData(Option lastCheckpoint } private JavaRDD toRDD(OffsetRange[] offsetRanges) { - return KafkaUtils.createRDD(sparkContext, + JavaRDD jsonStringRDD = KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), offsetRanges, LocationStrategies.PreferConsistent()) - .filter(x -> !StringUtils.isNullOrEmpty((String)x.value())) + .filter(x -> !StringUtils.isNullOrEmpty((String) x.value())) .map(x -> x.value().toString()); + return postProcess(jsonStringRDD); + } + + private JavaRDD postProcess(JavaRDD jsonStringRDD) { + String postProcessorClassName = this.props.getString(KafkaOffsetGen.Config.JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), null); + // no processor, do nothing + if (StringUtils.isNullOrEmpty(postProcessorClassName)) { + return jsonStringRDD; + } + + JsonKafkaSourcePostProcessor processor; + try { + processor = UtilHelpers.createJsonKafkaSourcePostProcessor(postProcessorClassName, this.props); + } catch (IOException e) { + throw new HoodieSourcePostProcessException("Could not init " + postProcessorClassName, e); + } + + return processor.process(jsonStringRDD); } @Override diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java index edf0d85bb32a0..564c5e2058453 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/KafkaOffsetGen.java @@ -189,6 +189,12 @@ public static class Config { .defaultValue(KafkaResetOffsetStrategies.LATEST) .withDocumentation("Kafka consumer strategy for reading data."); + public static final ConfigProperty JSON_KAFKA_PROCESSOR_CLASS_OPT = ConfigProperty + .key("hoodie.deltastreamer.source.json.kafka.processor.class") + .noDefaultValue() + .withDocumentation("Json kafka source post processor class name, post process data after consuming from" + + "source and before giving it to deltastreamer."); + public static final String KAFKA_CHECKPOINT_TYPE_TIMESTAMP = "timestamp"; } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/ChainedJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/ChainedJsonKafkaSourcePostProcessor.java new file mode 100644 index 0000000000000..827325a0bf559 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/ChainedJsonKafkaSourcePostProcessor.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.processor; + +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.spark.api.java.JavaRDD; + +import java.util.List; + +/** + * A {@link JsonKafkaSourcePostProcessor} to chain other {@link JsonKafkaSourcePostProcessor}s and apply sequentially. + */ +public class ChainedJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor { + + private final List processors; + + public ChainedJsonKafkaSourcePostProcessor(List processors, TypedProperties props) { + super(props); + this.processors = processors; + } + + @Override + public JavaRDD process(JavaRDD inputJsonRecords) { + JavaRDD targetRDD = inputJsonRecords; + for (JsonKafkaSourcePostProcessor processor : processors) { + targetRDD = processor.process(targetRDD); + } + return targetRDD; + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/JsonKafkaSourcePostProcessor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/JsonKafkaSourcePostProcessor.java new file mode 100644 index 0000000000000..7756dc5781481 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/processor/JsonKafkaSourcePostProcessor.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities.sources.processor; + +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.spark.api.java.JavaRDD; + +import scala.Serializable; + +/** + * Base class for Json kafka source post processor. User can define their own processor that extends this class to do + * some post process on the incoming json string records before the records are converted to DataSet. + */ +public abstract class JsonKafkaSourcePostProcessor implements Serializable { + + protected TypedProperties props; + + public JsonKafkaSourcePostProcessor(TypedProperties props) { + this.props = props; + } + + public abstract JavaRDD process(JavaRDD inputJsonRecords); +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaPostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaPostProcessor.java new file mode 100644 index 0000000000000..2f8d3eee7b62b --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaPostProcessor.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.utilities; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.utilities.schema.SchemaPostProcessor; + +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.spark.api.java.JavaSparkContext; + +public class DummySchemaPostProcessor extends SchemaPostProcessor { + + public DummySchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { + super(props, jssc); + } + + @Override + public Schema processSchema(Schema schema) { + return SchemaBuilder.record("test").fields().optionalString("testString").endRecord(); + } + +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaProvider.java index 2055406996f35..1ec3eb2f911f5 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaProvider.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/DummySchemaProvider.java @@ -18,9 +18,10 @@ package org.apache.hudi.utilities; -import org.apache.avro.Schema; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.Schema; import org.apache.spark.api.java.JavaSparkContext; public class DummySchemaProvider extends SchemaProvider { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java index 90c0149b0906c..22dca7bd6389f 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SparkAvroSchemaProvider.java @@ -18,9 +18,10 @@ package org.apache.hudi.utilities; -import org.apache.avro.Schema; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.avro.Schema; import org.apache.spark.api.java.JavaSparkContext; public class SparkAvroSchemaProvider extends SchemaProvider { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java index b9e9282923853..d228d87446df7 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestSchemaPostProcessor.java @@ -19,18 +19,19 @@ package org.apache.hudi.utilities; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.utilities.exception.HoodieSchemaPostProcessException; +import org.apache.hudi.utilities.schema.DeleteSupportSchemaPostProcessor; +import org.apache.hudi.utilities.schema.DropColumnSchemaPostProcessor; import org.apache.hudi.utilities.schema.SchemaPostProcessor; import org.apache.hudi.utilities.schema.SchemaPostProcessor.Config; import org.apache.hudi.utilities.schema.SchemaProvider; import org.apache.hudi.utilities.schema.SparkAvroPostProcessor; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; +import org.apache.hudi.utilities.transform.FlatteningTransformer; import org.apache.avro.Schema; import org.apache.avro.Schema.Type; -import org.apache.avro.SchemaBuilder; - -import org.apache.hudi.utilities.transform.FlatteningTransformer; -import org.apache.spark.api.java.JavaSparkContext; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -39,16 +40,20 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; public class TestSchemaPostProcessor extends UtilitiesTestBase { - private TypedProperties properties = new TypedProperties(); + private final TypedProperties properties = new TypedProperties(); - private static String ORIGINAL_SCHEMA = "{\"name\":\"t3_biz_operation_t_driver\",\"type\":\"record\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"null\",\"string\"],\"default\":null}," - + "{\"name\":\"ums_ts_\",\"type\":[\"null\",\"string\"],\"default\":null}]}"; + private static final String ORIGINAL_SCHEMA = "{\"type\":\"record\",\"name\":\"tripUberRec\",\"fields\":" + + "[{\"name\":\"timestamp\",\"type\":\"long\"},{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\"," + + "\"type\":\"string\"},{\"name\":\"driver\",\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"}]}"; - private static String RESULT_SCHEMA = "{\"type\":\"record\",\"name\":\"hoodie_source\",\"namespace\":\"hoodie.source\",\"fields\":[{\"name\":\"ums_id_\",\"type\":[\"null\",\"string\"]," - + "\"default\":null},{\"name\":\"ums_ts_\",\"type\":[\"null\",\"string\"],\"default\":null}]}"; + private static final String RESULT_SCHEMA = "{\"type\":\"record\",\"name\":\"hoodie_source\"," + + "\"namespace\":\"hoodie.source\",\"fields\":[{\"name\":\"timestamp\",\"type\":\"long\"}," + + "{\"name\":\"_row_key\",\"type\":\"string\"},{\"name\":\"rider\",\"type\":\"string\"},{\"name\":\"driver\"," + + "\"type\":\"string\"},{\"name\":\"fare\",\"type\":\"double\"}]}"; @Test public void testPostProcessor() throws IOException { @@ -83,21 +88,66 @@ public void testSparkAvro() throws IOException { } @Test - public void testSparkAvroSchema() throws IOException { - SparkAvroPostProcessor processor = new SparkAvroPostProcessor(properties, null); + public void testDeleteSupport() { + DeleteSupportSchemaPostProcessor processor = new DeleteSupportSchemaPostProcessor(properties, null); Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); - assertEquals(processor.processSchema(schema).toString(), RESULT_SCHEMA); + Schema targetSchema = processor.processSchema(schema); + assertNotNull(targetSchema.getField("_hoodie_is_deleted")); + } + + @Test + public void testChainedSchemaPostProcessor() { + // DeleteSupportSchemaPostProcessor first, DummySchemaPostProcessor second + properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, + "org.apache.hudi.utilities.schema.DeleteSupportSchemaPostProcessor,org.apache.hudi.utilities.DummySchemaPostProcessor"); + + SchemaPostProcessor processor = UtilHelpers.createSchemaPostProcessor(properties.getString(Config.SCHEMA_POST_PROCESSOR_PROP), properties, jsc); + Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); + Schema targetSchema = processor.processSchema(schema); + + assertNull(targetSchema.getField("_row_key")); + assertNull(targetSchema.getField("_hoodie_is_deleted")); + assertNotNull(targetSchema.getField("testString")); + + // DummySchemaPostProcessor first, DeleteSupportSchemaPostProcessor second + properties.put(Config.SCHEMA_POST_PROCESSOR_PROP, + "org.apache.hudi.utilities.DummySchemaPostProcessor,org.apache.hudi.utilities.schema.DeleteSupportSchemaPostProcessor"); + + processor = UtilHelpers.createSchemaPostProcessor(properties.getString(Config.SCHEMA_POST_PROCESSOR_PROP), properties, jsc); + schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); + targetSchema = processor.processSchema(schema); + + assertNull(targetSchema.getField("_row_key")); + assertNotNull(targetSchema.getField("_hoodie_is_deleted")); + assertNotNull(targetSchema.getField("testString")); } - public static class DummySchemaPostProcessor extends SchemaPostProcessor { + @Test + public void testDeleteColumn() { + // remove column ums_id_ from source schema + properties.put(DropColumnSchemaPostProcessor.Config.DELETE_COLUMN_POST_PROCESSOR_COLUMN_PROP, "rider"); + DropColumnSchemaPostProcessor processor = new DropColumnSchemaPostProcessor(properties, null); + Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); + Schema targetSchema = processor.processSchema(schema); + + assertNull(targetSchema.getField("rider")); + assertNotNull(targetSchema.getField("_row_key")); + } + + @Test + public void testDeleteColumnThrows() { + // remove all columns from source schema + properties.put(DropColumnSchemaPostProcessor.Config.DELETE_COLUMN_POST_PROCESSOR_COLUMN_PROP, "timestamp,_row_key,rider,driver,fare"); + DropColumnSchemaPostProcessor processor = new DropColumnSchemaPostProcessor(properties, null); + Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); - public DummySchemaPostProcessor(TypedProperties props, JavaSparkContext jssc) { - super(props, jssc); - } + Assertions.assertThrows(HoodieSchemaPostProcessException.class, () -> processor.processSchema(schema)); + } - @Override - public Schema processSchema(Schema schema) { - return SchemaBuilder.record("test").fields().optionalString("testString").endRecord(); - } + @Test + public void testSparkAvroSchema() throws IOException { + SparkAvroPostProcessor processor = new SparkAvroPostProcessor(properties, null); + Schema schema = new Schema.Parser().parse(ORIGINAL_SCHEMA); + assertEquals(processor.processSchema(schema).toString(), RESULT_SCHEMA); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java index a57be62461d45..9a62c14e5caa9 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java @@ -249,14 +249,17 @@ public List createUpsertRecords(Path srcFolder) throws ParseExcep long startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime() / 1000; List records = new ArrayList(); // 10 for update + HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); for (long recordNum = 0; recordNum < 11; recordNum++) { - records.add(new HoodieTestDataGenerator().generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, - "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); + records.add( + dataGen.generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, + "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } // 4 for insert for (long recordNum = 96; recordNum < 100; recordNum++) { - records.add(new HoodieTestDataGenerator().generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, - "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); + records.add( + dataGen.generateGenericRecord(Long.toString(recordNum), "0", "rider-upsert-" + recordNum, + "driver-upsert" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); } try (ParquetWriter writer = AvroParquetWriter.builder(srcFile) .withSchema(HoodieTestDataGenerator.AVRO_SCHEMA).withConf(HoodieTestUtils.getDefaultHadoopConf()).build()) { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 1c80896586515..aa233d4e37d3e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -70,6 +70,7 @@ import org.apache.hudi.utilities.sources.ParquetDFSSource; import org.apache.hudi.utilities.sources.SqlSource; import org.apache.hudi.utilities.sources.TestDataSource; +import org.apache.hudi.utilities.sources.TestParquetDFSSourceEmptyBatch; import org.apache.hudi.utilities.testutils.JdbcTestUtils; import org.apache.hudi.utilities.testutils.UtilitiesTestBase; import org.apache.hudi.utilities.testutils.sources.DistributedTestDataSource; @@ -130,6 +131,7 @@ import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -1420,15 +1422,34 @@ private void prepareParquetDFSSource(boolean useSchemaProvider, boolean hasTrans } private void testParquetDFSSource(boolean useSchemaProvider, List transformerClassNames) throws Exception { + testParquetDFSSource(useSchemaProvider, transformerClassNames, false); + } + + private void testParquetDFSSource(boolean useSchemaProvider, List transformerClassNames, boolean testEmptyBatch) throws Exception { prepareParquetDFSSource(useSchemaProvider, transformerClassNames != null); String tableBasePath = dfsBasePath + "/test_parquet_table" + testNum; HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer( - TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, ParquetDFSSource.class.getName(), + TestHelpers.makeConfig(tableBasePath, WriteOperationType.INSERT, testEmptyBatch ? TestParquetDFSSourceEmptyBatch.class.getName() + : ParquetDFSSource.class.getName(), transformerClassNames, PROPS_FILENAME_TEST_PARQUET, false, useSchemaProvider, 100000, false, null, null, "timestamp", null), jsc); deltaStreamer.sync(); TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext); testNum++; + + if (testEmptyBatch) { + prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "2.parquet", false, null, null); + // parquet source to return empty batch + TestParquetDFSSourceEmptyBatch.returnEmptyBatch = true; + deltaStreamer.sync(); + // since we mimic'ed empty batch, total records should be same as first sync(). + TestHelpers.assertRecordCount(PARQUET_NUM_RECORDS, tableBasePath + "/*/*.parquet", sqlContext); + HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build(); + + // validate table schema fetches valid schema from last but one commit. + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); + assertNotEquals(tableSchemaResolver.getTableAvroSchema(), Schema.create(Schema.Type.NULL).toString()); + } } private void testORCDFSSource(boolean useSchemaProvider, List transformerClassNames) throws Exception { @@ -1584,6 +1605,11 @@ public void testParquetDFSSourceWithoutSchemaProviderAndNoTransformer() throws E testParquetDFSSource(false, null); } + @Test + public void testParquetDFSSourceForEmptyBatch() throws Exception { + testParquetDFSSource(false, null, true); + } + @Test public void testParquetDFSSourceWithoutSchemaProviderAndTransformer() throws Exception { testParquetDFSSource(false, Collections.singletonList(TripsWithDistanceTransformer.class.getName())); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java index ff0a19273f376..87f1774e02d2e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSource.java @@ -62,12 +62,12 @@ */ public class TestJsonKafkaSource extends SparkClientFunctionalTestHarness { - private static final String TEST_TOPIC_PREFIX = "hoodie_test_"; + protected static final String TEST_TOPIC_PREFIX = "hoodie_test_"; private static final URL SCHEMA_FILE_URL = TestJsonKafkaSource.class.getClassLoader().getResource("delta-streamer-config/source.avsc"); - private static KafkaTestUtils testUtils; + protected static KafkaTestUtils testUtils; - private final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); - private FilebasedSchemaProvider schemaProvider; + protected final HoodieDeltaStreamerMetrics metrics = mock(HoodieDeltaStreamerMetrics.class); + protected FilebasedSchemaProvider schemaProvider; @BeforeAll public static void initClass() throws Exception { @@ -88,7 +88,7 @@ public void init() throws Exception { schemaProvider = new FilebasedSchemaProvider(props, jsc()); } - private TypedProperties createPropsForJsonSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { + protected TypedProperties createPropsForJsonSource(String topic, Long maxEventsToReadFromKafkaSource, String resetStrategy) { TypedProperties props = new TypedProperties(); props.setProperty("hoodie.deltastreamer.source.kafka.topic", topic); props.setProperty("bootstrap.servers", testUtils.brokerAddress()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.java new file mode 100644 index 0000000000000..bd150ed29be38 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestJsonKafkaSourcePostProcessor.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.utilities.sources; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.utilities.deltastreamer.SourceFormatAdapter; +import org.apache.hudi.utilities.exception.HoodieSourcePostProcessException; +import org.apache.hudi.utilities.sources.processor.JsonKafkaSourcePostProcessor; + +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.api.java.JavaRDD; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Objects; + +import static org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.Config.JSON_KAFKA_PROCESSOR_CLASS_OPT; +import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.jsonifyRecords; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + +public class TestJsonKafkaSourcePostProcessor extends TestJsonKafkaSource { + + @Test + public void testNoPostProcessor() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testNoPostProcessor"; + testUtils.createTopic(topic, 2); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); + + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); + + assertEquals(900, fetch1.getBatch().get().count()); + } + + @Test + public void testSampleJsonKafkaSourcePostProcessor() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testSampleJsonKafkaSourcePostProcessor"; + testUtils.createTopic(topic, 2); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + + // processor class name setup + props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), SampleJsonKafkaSourcePostProcessor.class.getName()); + + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); + + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); + + assertNotEquals(900, fetch1.getBatch().get().count()); + } + + @Test + public void testInvalidJsonKafkaSourcePostProcessor() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testInvalidJsonKafkaSourcePostProcessor"; + testUtils.createTopic(topic, 2); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + + // processor class name setup + props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), "InvalidJsonKafkaSourcePostProcessor"); + + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + + Assertions.assertThrows(HoodieSourcePostProcessException.class, + () -> kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900)); + } + + @Test + public void testChainedJsonKafkaSourcePostProcessor() { + // topic setup. + final String topic = TEST_TOPIC_PREFIX + "testChainedJsonKafkaSourcePostProcessor"; + testUtils.createTopic(topic, 2); + + HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(); + TypedProperties props = createPropsForJsonSource(topic, null, "earliest"); + + // processor class name setup + props.setProperty(JSON_KAFKA_PROCESSOR_CLASS_OPT.key(), SampleJsonKafkaSourcePostProcessor.class.getName() + + "," + DummyJsonKafkaSourcePostProcessor.class.getName()); + + Source jsonSource = new JsonKafkaSource(props, jsc(), spark(), schemaProvider, metrics); + SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); + + testUtils.sendMessages(topic, jsonifyRecords(dataGenerator.generateInserts("000", 1000))); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); + + assertEquals(0, fetch1.getBatch().get().count()); + } + + /** + * JsonKafkaSourcePostProcessor that return a sub RDD of the incoming data which get the data from incoming data using + * {org.apache.spark.api.java.JavaRDD#sample(boolean, double, long)} method. + */ + public static class SampleJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor { + + public SampleJsonKafkaSourcePostProcessor(TypedProperties props) { + super(props); + } + + @Override + public JavaRDD process(JavaRDD inputJsonRecords) { + return inputJsonRecords.sample(false, 0.5); + } + } + + public static class DummyJsonKafkaSourcePostProcessor extends JsonKafkaSourcePostProcessor { + public DummyJsonKafkaSourcePostProcessor(TypedProperties props) { + super(props); + } + + @Override + public JavaRDD process(JavaRDD inputJsonRecords) { + // return empty RDD + return inputJsonRecords.map(x -> "").filter(x -> !Objects.equals(x, "")); + } + } + +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java new file mode 100644 index 0000000000000..3129e91a9d3e0 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestParquetDFSSourceEmptyBatch.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.utilities.sources; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +public class TestParquetDFSSourceEmptyBatch extends ParquetDFSSource { + + public static boolean returnEmptyBatch; + + public TestParquetDFSSourceEmptyBatch(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + } + + @Override + public Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { + Pair>, String> toReturn = super.fetchNextBatch(lastCkptStr, sourceLimit); + if (returnEmptyBatch) { + return Pair.of(Option.empty(), toReturn.getRight()); + } + return toReturn; + } +} diff --git a/pom.xml b/pom.xml index 1b28ae1bb9a49..33bd112a1bfe3 100644 --- a/pom.xml +++ b/pom.xml @@ -51,9 +51,6 @@ packaging/hudi-utilities-bundle packaging/hudi-timeline-server-bundle packaging/hudi-trino-bundle - docker/hoodie/hadoop - hudi-integ-test - packaging/hudi-integ-test-bundle hudi-examples hudi-flink hudi-kafka-connect @@ -1187,6 +1184,9 @@ true + + true + @@ -1357,6 +1357,16 @@ integration-tests + + + integration-tests + + + + docker/hoodie/hadoop + hudi-integ-test + packaging/hudi-integ-test-bundle + true true @@ -1578,13 +1588,25 @@ - spark2 + + hudi-spark-datasource/hudi-spark2 + hudi-spark-datasource/hudi-spark2-common + + + true + + spark2 + + !disabled + + spark3 + 3.2.1 ${spark3.version} ${spark3.version} ${scala12.version} @@ -1603,6 +1625,10 @@ true true + + hudi-spark-datasource/hudi-spark3 + hudi-spark-datasource/hudi-spark3-common + spark3 @@ -1629,6 +1655,10 @@ true true + + hudi-spark-datasource/hudi-spark3.1.x + hudi-spark-datasource/hudi-spark3-common + spark3.1.x diff --git a/rfc/README.md b/rfc/README.md index a9587d1d79cf3..0b3142e8bd6e5 100644 --- a/rfc/README.md +++ b/rfc/README.md @@ -70,3 +70,4 @@ The list of all RFCs can be found here. | 44 | [Hudi Connector for Presto](./rfc-44/rfc-44.md) | `UNDER REVIEW` | | 45 | [Asynchronous Metadata Indexing](./rfc-45/rfc-45.md) | `UNDER REVIEW` | | 46 | [Optimizing Record Payload Handling](./rfc-46/rfc-46.md) | `UNDER REVIEW` | +| 47 | [Add Call Produce Command for Spark SQL](./rfc-47/rfc-47.md) | `UNDER REVIEW` | diff --git a/rfc/rfc-27/rfc-27.md b/rfc/rfc-27/rfc-27.md index 3b00af7c140a0..1454dce0d4b5f 100644 --- a/rfc/rfc-27/rfc-27.md +++ b/rfc/rfc-27/rfc-27.md @@ -34,19 +34,20 @@ JIRA: https://issues.apache.org/jira/browse/HUDI-1822 ## Abstract -Query engines typically scan large amounts of irrelevant data for query planning and execution. Some workarounds are -available to reduce amount of irrelevant data scanned. These include +Query engines typically scan large amounts of data for query planning and execution. Few data skipping strategies are +available to reduce the amount of data scanned, like + - Partition pruning -- File pruning
- - Some data file formats contain metadata including range information for certain columns (for parquet, this metadata - is stored in footer). - - As part of query planning, all range information from data files is read. - - Irrelevant data files are then pruned based on predicates and available range information - -Partition pruning typically puts the burden on users to select partitions where the data may exist. File pruning approach - is expensive and does not scale if there are large number of partitions and data files to be scanned. So we propose a - new solution to store additional information as part of Hudi metadata table to implement data skipping index. The - goals of data skipping index is to provide: + - User has to select the partitions to narrow down the data to be scanned for the query. +- File pruning + - Some data file formats contain metadata including range information for certain columns, like for parquet, this + metadata is stored in the file footer. As part of query planning, all range information from data files are loaded + and data files are then pruned based on the comparisons done for the query expression with the column range + information. + - This approach does not scale if there are a large number of partitions and data files to be scanned. + +We propose a new data skipping approach here for improving the query performance. to store additional information as +part of Hudi metadata table to implement data skipping index. The goals of data skipping index is to provide: - Global index: Users query for information they need without need for specifying partitions. Index can effectively find data files in the table. @@ -90,18 +91,18 @@ So, high level requirement for this column_stats partition is (pertaining to thi ### Storage format To cater to the above requirement, we plan to encode column name, partition path and file name to the keys in HFile. -Since HFile supports efficient range/prefix search, our look up should be very fast. +Since HFile supports efficient range/prefix search, our lookup should be very fast. ![Column Stats Partition](col_stats.png) We plan to generate unique and random and unique hash IDs for all 3 components -- ColumnID : +- ColumnIndexID : - base64(hash32(column name)) - on-disk size = 12bytes per col_stat per file -- PartitionID: +- PartitionIndexID: - base64(hash32(partition name)) - on-disk size = 12bytes per partition -- FileID: +- FileIndexID: - base64(hash128(file name)) - on-disk size = 24bytes per file @@ -127,8 +128,8 @@ We plan to generate unique and random and unique hash IDs for all 3 components Takes up larger space in-memory and on-disk compared to Sequential IDs. Theoretically, the compression ratio should be lesser compared to Sequential IDs. Key format in column_stats partition
-- [colId][PartitionId][FileId] -- [colId]+"agg"+[PartitionId] +- ColumnStatsIndexID = ColumnIndexID.append(PartitionIndexID).append(FileIndexID) +- ColumnStatsAggregateIndexID = ColumnIndexID.append(PartitionIndexID) First type will be used to store one entry per column per file. And second type will be used to store one aggregated entry per column per partition. This will be a fixed size key. Lookups don't have to search for ID delimiters as in the @@ -140,17 +141,17 @@ our use-case as we have chosen the key format consciously having this in mind. Given a list of columns and optionally partitions, return a list of matching file names. -1. We can do prefix search of [ColumnID] or [ColumnID][PartitionID] +1. We can do prefix search of [ColumnIndexID] or [ColumnIndexID][PartitionIndexID] - If both columnId and partitionIds are supplied, we will do range read of [colId][partitionId]. - If list of partitions not available as part of query, we will first look up [colId]+"agg" to do prefix search for partition level stats. Filter for those partitions which matches the predicates and then follow (1) as in previous line. 2. Fetch only interested entries for [colId][partitionId] list. -3. Will look up the stats and filter for matching FileIDs -4. Reverse lookup in Files partition to get FileID to FileName mapping. +3. Will look up the stats and filter for matching FileIndexIDs +4. Reverse lookup in Files partition to get FileIndexID to FileName mapping. Note: -As you could see here, reverse look up of FileId to fileName mapping has to go into "Files" partition to satisfy our requirement. +As you could see here, reverse look up of FileIndexID to fileName mapping has to go into "Files" partition to satisfy our requirement. So, "Files" partition will be added with additional entries of fileId to fileName mappings on the write path. #### Sharding: @@ -194,54 +195,75 @@ field will be used to detect the column stats payload record. Here is the schema . }, { + "doc": "Metadata Index of column statistics for all data files in the user table", "name": "ColumnStatsMetadata", - "doc": "Contains information about column statistics for all data files in the table", "type": [ "null", { - "type": "record", + "doc": "Data file column statistics", "name": "HoodieColumnStats", + "type": "record", "fields": [ { - "name": "rangeLow", + "doc": "File name for which this column statistics applies", + "name": "fileName", "type": [ "null", - "bytes" - ], - "doc": "Low end of the range. For now, this is a String. Based on main data table schema, we can convert it to appropriate type" + "string" + ] }, { - "name": "rangeHigh", + "doc": "Minimum value in the range. Based on user data table schema, we can convert this to appropriate type", + "name": "minValue", "type": [ "null", - "bytes" - ], - "doc": "High end of the range. For now, this is a String. Based on main data table schema, we can convert it to appropriate type" + "string" + ] }, { - "name":"total_values", - "type":["long", "null"], - "doc" : "Stores total values for this column in the resepective data file" - }, + "doc": "Maximum value in the range. Based on user data table schema, we can convert it to appropriate type", + "name": "maxValue", + "type": [ + "null", + "string" + ] + }, { - "name":"total_nulls", - "type":["long", "null"], - "doc" : "Stores total null values for this column in the resepective data file" - }, + "doc": "Total count of values", + "name": "valueCount", + "type": [ + "null", + "long" + ] + }, { - "name":"total_nans", - "type":["long", "null"], - "doc" : "Stores total Nan values for this column in the resepective data file" - }, + "doc": "Total count of null values", + "name": "nullCount", + "type": [ + "null", + "long" + ] + }, + { + "doc": "Total storage size on disk", + "name": "totalSize", + "type": [ + "null", + "long" + ] + }, { - "name":"total_size_on_disk", - "type":["long", "null"], - "doc" : "Stores total size occupied by this column on disk corresponding to the resepective data file" - }, + "doc": "Total uncompressed storage size on disk", + "name": "totalUncompressedSize", + "type": [ + "null", + "long" + ] + }, { + "doc": "Column range entry valid/deleted flag", "name": "isDeleted", - "type": "boolean", - "doc": "True if this file has been deleted" + "type": "boolean" } ] } @@ -254,7 +276,7 @@ encoded string as discussed earlier. ``` key = base64_encode(hash64(column name) + hash64(partition name) + hash128(file path)) -key = base64_encode(hash64(column name) + "agg" + hash64(partition name)) +key = base64_encode(hash64(column name) + hash64(partition name)) ``` While Hash based IDs have quite a few desirable properties in the context of Hudi index lookups, there is an impact @@ -265,13 +287,13 @@ Let's walk through the writer flow to update column_stats partition in metadata 1. Files partition - prepare records for adding // just calling out whats required in the context of column_stats partition. General files partition will be updated as usual to store file listing information. - - FileID => FileName mapping entries - - PartitionID => PartitionName entry, if not already exists + - FileIndexID => FileName mapping entries + - PartitionIndexID => PartitionName entry, if not already exists - Since these IDs are hash based IDs, no look up of prior usages is required here. If not, we need to know what was the last assigned ID and then go about assigning new incremental/sequential IDs, which slows down the performance significantly 2. Column_stats partition - prepare records for adding - - [ColumnID][PartitionID][FileID] => ColumnStat - - [ColumnId]"agg"[PartitionId] => ColumnStat + - [ColumnIndexID][PartitionIndexID][FileIndexID] => ColumnStat + - [ColumnIndexID]"agg"[PartitionIndexID] => ColumnStat - This involves reading the base file footers to fetch min max and other stats to populate values for the record. d. Commit all these records to metadata table. diff --git a/rfc/rfc-42/basic_bucket_hashing.png b/rfc/rfc-42/basic_bucket_hashing.png new file mode 100644 index 0000000000000..3cdfcef5f6b03 Binary files /dev/null and b/rfc/rfc-42/basic_bucket_hashing.png differ diff --git a/rfc/rfc-42/bucket_resizing.png b/rfc/rfc-42/bucket_resizing.png new file mode 100644 index 0000000000000..ce41be9b6c9a8 Binary files /dev/null and b/rfc/rfc-42/bucket_resizing.png differ diff --git a/rfc/rfc-42/bucket_resizing_virtual_log_file.png b/rfc/rfc-42/bucket_resizing_virtual_log_file.png new file mode 100644 index 0000000000000..ba0efe76de01a Binary files /dev/null and b/rfc/rfc-42/bucket_resizing_virtual_log_file.png differ diff --git a/rfc/rfc-42/consistent_hashing.png b/rfc/rfc-42/consistent_hashing.png new file mode 100644 index 0000000000000..278a8981b05dc Binary files /dev/null and b/rfc/rfc-42/consistent_hashing.png differ diff --git a/rfc/rfc-42/rfc-42.md b/rfc/rfc-42/rfc-42.md new file mode 100644 index 0000000000000..97f4146f9a711 --- /dev/null +++ b/rfc/rfc-42/rfc-42.md @@ -0,0 +1,230 @@ + +# RFC-42: Consistent Hashing Index for Dynamic Bucket Number + + +## Proposers + +- @HuberyLee +- @hujincalrin +- @stream2000 +- @YuweiXiao + +## Approvers + + - @garyli1019 + - @leesf + - @vinothchandar + +## Status + +JIRA: [HUDI-3000](https://issues.apache.org/jira/browse/HUDI-3000) + +> Please keep the status updated in `rfc/README.md`. + +## Abstract + +Hudi supports `Upsert` operation to de-duplicate records in a table, which depends on indexing schemes to perform record location lookup. +Among many index options, bucket index (in progress, [RFC-29](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+29%3A+Hash+Index)) achieves promising Upsert performance, around ~3x improvement on throughput compared to using Bloom Filter. +However, it requires pre-configure a fixed bucket number and cannot be changed afterwards. +Combined with the design of one-one mapping between hash buckets and file groups, hudi tables with bucket index have some practical issues, such as data skew and unlimited file group size, which now can only be resolved by resetting a suitable bucket number through re-writing the whole table. + +This proposal wants to tackle these problems by introducing **Consistent Hashing Index**. +It achieves bucket resizing by splitting or merging several local buckets (i.e., only large file groups) while leaving most buckets untouched. +This feature allows us to adjust bucket number dynamically in a background service with minimal impacts on downstream systems relying on Hudi. +For example, concurrent readers and writers are not blocked during the resizing. + + +## Background + +Hudi supports the primary key concept from day one through a write operation called `Upsert`. +To correctly enforce the uniqueness of keys, `Upsert` performs indexing to locate data files where every record belongs. +One of the index implementations is `Bucket Index`, shown in the following figure. +It distributes records to buckets using a hash function, and each bucket corresponds to a single file group (i.e., one-one mapping). +This simple yet effective design reduce the time complexity of the key lookup to constant time (i.e., hash function computation), bringing good write performance. + +![bucket index](./basic_bucket_hashing.png) + +However, there are also some limitions. +As described in [RFC-29](https://cwiki.apache.org/confluence/display/HUDI/RFC+-+29%3A+Hash+Index), the one-one mapping between buckets and file groups may cause data skew and doesn't scale well. +One solution to address these problems is allowing one bucket to have multiple file groups, which in turn requires indexing to be performed inside each bucket. + +Another solution, that this proposal chooses, is to adjust bucket number dynamically based on Consistent Hashing. +In contrast to a standard re-hashing process, which needs shuffling of the whole table, Consistent Hashing constrains the re-hashing process to touch several local buckets (e.g, only large file groups). +The figure below shows a basic Consistent Hashing algorithm: + +![consistent hashing index](./consistent_hashing.png) + +Hash value is obtained by computing `Hash(v) % 0xFFFF`, which falls into a pre-defined range (i.e., [0, 0xFFFF] in the figure). +Then a range mapping is applied to the hash value to get the final bucket. +The figure also demonstrates a local bucket split process, where Bucket #2 is split into two children buckets and increases the total number of buckets by one. +Compared to a traditional hashing scheme, Consistent Hashing introduces an extra range mapping layer, linking hash values and buckets. + + + +## Implementation + +The design consideration and implementation will mostly follow the current Bucket Index: + +1. Hashing happens at partition-level, i.e., each partition is managed independently and will be divided into N buckets. +2. One-one mapping between hash buckets and file groups. +3. The assignment of records to buckets fits into a standard write path by overriding the `tag location` process. + +To further support Consistent Hashing, we also need to consider: + +1. Hashing metadata. Information, such as range mapping between `hash_value` and `buckets`, should be maintained and persisted as files. +2. Implementation details about bucket resizing (splitting & merging): resizing criteria and how to integrate it as a Hudi service. +3. Concurrent writers and readers: ensure a non-blocking resizing process. + +To enable different dynamic resizing strategies (i.e., other than consistent hashing), a flexible framework will be implemented for future extension. +For example, in the Bucket Index RFC-29, *extendible hashing* is mentioned for solving the limitations of bucket index. + +For end-users, a new index option `BUCKET_INDEX_ENGINE` will be provided as an advanced configuration of `BUCKET_INDEX`. +One can choose different hashing options according to their use case, e.g., `SIMPLE_HASHING`, `CONSISTENT_HASHING` or `EXTENDIBLE_HASHING`. + +When a dynamic resizing strategy is selected, the bucket number option (`BUCKET_INDEX_NUM_BUCKETS`) serves as an initial setup and requires no hyper-parameter tuning, as it will auto adjust in the course of data ingestion. + +### Hashing Metadata + +The hashing metadata will be persisted as files named as `.hashing_meta` for each partition as we manage hashing for each partition independently. +It stores in `.hoodie/.hashing_meta/partition/` directory and contains the following information in a readable encoding (e.g., json): +``` +# reserved for metadata schema evolution +version: 1 +partition_path: xxx +# last instant that updates the metadata +instant: xxx +# total number of bucket in current partition +num_buckets: xxx +bucket_mappings: [ + # hash_values are positive integer number, i.e., from 0 to 0x7fffffff (Integer.MAX_VALUE) + # All hash_values divide the hash space into a group of continuous hash rangs, serving the range mapping step + {, file_group} + {, file_group} + ... + {0x7FFFFFFF, file_group} +] +# other necessary or useful fields +# e.g., statistics about hash value distribution (maybe useful for bucket resizing) +... +``` + +Assuming thousands of buckets case, the metadata file is around several MB in size. +Only three operations will modify the hashing metadata: + +- Initial write to a partition: a fresh new hashing metadata will be created with timestamp `00000000000000`, with default equally divided range mapping and randomly generated file group UUID. +- Resizing: a new version of hashing metadata will be created when the resizing completes, named using the timestamp of the resizing operation. +- Clean service: removes outdated metadata files in the same way as removing old file groups. + +Though storing hashing metadata in the `.hoodie` path is a straightforward solution, it should be put into hudi's metadata table ultimately. +And the clean service is no loger necessary. +Old version hashing metadata will also be cleanup automatically since the metadata table itself is a hudi table. +### Bucket Resizing (Splitting & Merging) + +Considering there is a semantic similarity between bucket resizing and clustering (i.e., re-organizing small data files), this proposal plans to integrate the resizing process as a subtask into the clustering service. +The trigger condition for resizing directly depends on the file size, where small files will be merged and large files will be split. + +**For merging files**, we require that the buckets should be adjacent to each other in terms of their hash ranges so that the output bucket has only one continuous hash range. +Though it is not required in a standard Consistent Hashing algorithm, fragmentations in hash ranges may cause extra complexity for the splitting process in our case. + +**For splitting files**, a split point (i.e., hash ranges for the output buckets) should be decided: + +- A simple policy would be split in the range middle, but it may produce uneven data files. +In an extreme case, splitting may produce one file with all data and one file with no data. +- Another policy is to find a split point that evenly dispatches records into children buckets. +It requires knowledge about the hash value distribution of the original buckets. + +In our implementation, *we will first stick to the first simple one*, as buckets will finally converge to a balanced distribution after multiple rounds of resizing processes. +Of course, a pluggable implementation will be kept for extensibility so that users can choose different available policies. + +All updates related to the hash metadata will be first recorded in the clustering plan, and then be reflected in partitions' hashing metadata when clustering finishes. +The plan is generated and persisted in files during the scheduling process, which is protected by a table-level lock for a consistent table view. + +### Concurrent Writer & Reader + +Concurrent updates to file groups that are under clustering are not supported in Hudi, since records location change during the re-organizing. +For example, a clustering process is scheduled to combine three parquet files A, B, C into two parquet files E, F with a more compact storage layout. +During the clustering, a writer comes in and plan to update a record that originally belongs to the parquet file A. +There are two choices the writer can make: + +- Only update the old file (i.e. parquet file A), but the update will lost once the clustering finishes because of the *replacecommit*'s shadowing. +- Besides parquet file A, also update new files (i.e. parquet file E or F). +However, the record's new location can only be determined until the clustering finishes, which means the writer will be blocked as it has to wait for the location information. + +For tables using Bucket Index, the above conflicting condition can be avoided because record locations are calculated by the hash algorithm. +So even before the clustering finishes, the writer can calculate record locations as long as it knows the latest hash algorithm (in our Consistent Hashing case, it is stored in the clustering plan). + +#### Dual write solution + +![bucket resizing](bucket_resizing.png) + +The figure above shows a concurrent write process during the bucket resizing process (i.e., splitting). +Dual-write is activated for buckets that are splitting. +Each incoming record will be written to log files of both the old file group (i.e., FG1) and children file groups (i.e., FG1.1 or FG1.2). + +With this dual-write scheme, no modification is required for concurrent readers. +The default behaviour of a reader is: + +- Query old files before the resizing process finishes. +- Query new files when the resizing process finishes. + +As the writer put incoming records in both new and old files, readers will always see the update-to-date data. + +#### Virtual log file solution + +![bucket resizing using virtual log file](bucket_resizing_virtual_log_file.png) + +Instead of writing records to both old and children file groups physically, we could route records to children file groups only and create an 'empty' virtual log file in the old file group. +The virtual log file contains the necessary information to reference log files in the children file groups (i.e., red lines in the above figure). +It enables readers to see the latest completed writes by merging children log files with the base file of the old file group. + +The virtual file solution avoids extra write workload and minimizes the impact of resizing on the write performance. +However, it adds some work on the read path in order to understand the virtual log file. + +### Performance + +Hudi table using Consistent Hashing should obtain a better write throughput than using Bloom Filter or HBase Index, thanks to the elimination of costs about indexing and read/write amplification. +But the throughput is expected to be a little smaller than using Bucket Index, as Consistent Hashing has a larger overhead on computing the bucket id, as shown in the following table. + +| | Consistent Hashing Index | Bucket Index | +|-------------------|--------------------------|--------------| +| lookup a record | log(N) | O(1) | +| resizing a bucket | K*O(1) | K*O(N) | + +, where N denotes the number of buckets and K represents the average cost of re-hashing one bucket. + +When the resizing of buckets is going on, the relevant writer that is on dual-write mode may drop ~50% in performance, since it needs to write records (as logs) twice into two file groups. +To alleviate the performance impact, we could control the resizing granularity: +- limit the number of involevd partitions per schedule. +- limit the total processing data size(e.g., 2G) per schedule. +- combination of above strategies and data size control has a higher priority. + +## Rollout/Adoption Plan + +There are no impacts to existing users since this is a fresh new index scheme, so no migrations/behaviour changes are required. + +## Test Plan + +* Unit tests of new modules will be provided. + +* Integration test with different cases will be included: + 1. Initializing of tables using Consistent Hashing. + 2. Multiple rounds of merging & splitting. + 2. Extreme data distribution case where most data fall into a small hash range. + 2. Concurrent writer & reader case. + 4. Concurrent updates to hash metadata (e.g., multiple async clusterings). +* Performance (i.e., write throughput) test result will be provided, comparing Consistent Hashing Index, Bucket Index and Bloom Filter. \ No newline at end of file diff --git a/rfc/rfc-46/rfc-46.md b/rfc/rfc-46/rfc-46.md index 8b0feff2343db..92aed5554de36 100644 --- a/rfc/rfc-46/rfc-46.md +++ b/rfc/rfc-46/rfc-46.md @@ -156,4 +156,4 @@ This refactoring will not be modifying any existing Hudi semantics other than th logical correctness of the many flows that will be affected by the refactoring we will rely on the existing set of test-suites. Nevertheless, we will run corresponding set of benchmarks stressing the flows being affected by the refactoring to validate -that there are considerable performance advantage of abandoning conversion into intermediate representation completely. \ No newline at end of file +that there are considerable performance advantage of abandoning conversion into intermediate representation completely. diff --git a/rfc/rfc-47/process.png b/rfc/rfc-47/process.png new file mode 100644 index 0000000000000..0553fea6f78c4 Binary files /dev/null and b/rfc/rfc-47/process.png differ diff --git a/rfc/rfc-47/rfc-47.md b/rfc/rfc-47/rfc-47.md new file mode 100644 index 0000000000000..f9017ec612511 --- /dev/null +++ b/rfc/rfc-47/rfc-47.md @@ -0,0 +1,111 @@ + + +# RFC-[47]: Add RFC for Add Call Produce Command for Spark SQL + + +## Proposers +- @forwardxu + +## Approvers + - @vinoth @Danny @Raymond + +## Status + +JIRA: [https://issues.apache.org/jira/browse/HUDI-3161](https://issues.apache.org/jira/browse/HUDI-3161) + +## Abstract + +We intend to implement Call Procedure Command for Spark SQL. + +## Background +The Call Procedure Command can solve the problems that DDL and DML functions cannot handle. What I can think of are the following 4 aspects: +- Commit management +- Metadata table management +- Table migration +- Optimization table(Such as Table services like clustering) + +## Implementation + +### Extended SQL for CALL Command Syntax +#### Named arguments +All procedure arguments are named. When passing arguments by name, arguments can be in any order and any optional argument can be omitted. +``` +CALL system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1, ... arg_name_n => arg_n) +``` +#### Positional arguments +When passing arguments by position, the arguments may be omitted if they are optional. +``` +CALL system.procedure_name(arg_1, arg_2, ... arg_n) +``` +*note:* The system here has no practical meaning, the complete procedure name is system.procedure_name. + +![](process.png) + +As shown in the figure above, The execution process of Call Command consists of two parts, SQL Parser and Procedure Run. + +### parse +In the sql parse stage, we will inject a HoodieSqlParser to spark sql which will parse our extended `CALL` syntax to LogicalPlan. If the HoodieSqlParser failed to parse the sql statement, Spark will route it to Spark SQL parser. So we just need to implement our extended syntax in `HoodieSqlParser`. + +### resolution +In the resolution stage, some Hudi resolution rules will be injected to Spark SQL to resolve our extended LogicalPlan to the resolve plan which is a command plan for `CALL`. + +### procedure#call +The `procedure#call` method will translate the logical plan to Hudi's API call. For example, the `ShowCommitsProcedure` will be translated to Hudi API for showing Hudi timeline's commits. + +### examples +Example 1: + +``` +call show_commits_metadata(table => 'test_hudi_table'); +``` + +| commit_time | action | partition | file_id | previous_commit | num_writes | num_inserts | num_deletes | num_update_writes | total_errors | total_log_blocks | total_corrupt_logblocks | total_rollback_blocks | total_log_records | total_updated_records_compacted | total_bytes_written| +|----------------- | ---- | ------------ |----------------------------------------| ---------------- |------------- |-------------|-------------|------------------ |------------- |---------------- |-------------------------|---------------------- |-------------------|------------------------------- |------------------- | +|20220109225319449 | commit | dt=2021-05-03 | d0073a12-085d-4f49-83e9-402947e7e90a-0 | null | 1 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 435349 | +|20220109225311742 | commit | dt=2021-05-02 | b3b32bac-8a44-4c4d-b433-0cb1bf620f23-0 | 20220109214830592| 1 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 435340 | +|20220109225301429 | commit | dt=2021-05-01 | 0d7298b3-6b55-4cff-8d7d-b0772358b78a-0 | 20220109214830592| 1 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 435340 | +|20220109214830592 | commit | dt=2021-05-01 | 0d7298b3-6b55-4cff-8d7d-b0772358b78a-0 | 20220109191631015| 0 | 0 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 432653 | +|20220109214830592 | commit | dt=2021-05-02 | b3b32bac-8a44-4c4d-b433-0cb1bf620f23-0 | 20220109191648181| 0 | 0 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 432653 | +|20220109191648181 | commit | dt=2021-05-02 | b3b32bac-8a44-4c4d-b433-0cb1bf620f23-0 | null | 1 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 435341 | +|20220109191631015 | commit | dt=2021-05-01 | 0d7298b3-6b55-4cff-8d7d-b0772358b78a-0 | null | 1 | 1 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 0 | 435341 | + +Time taken: 0.844 seconds, Fetched 7 row(s) + +Example 2: + +``` +call rollback_to_instant(table => 'test_hudi_table', instant_time => '20220109225319449'); +``` + +| rollback_result | +| :---------------| +| true | + +Time taken: 5.038 seconds, Fetched 1 row(s) + +### Spark SQL permissions and security +For security, please refer to the description of [Spark Security](https://spark.apache.org/docs/latest/security.html). + +## Rollout/Adoption Plan +This is a new feature can use Spark SQL works And does not depend on the Spark version. + +## Test Plan + +- [☑️️] Unit tests for this feature +- [✖️] Product integration tests +- [✖️] Benchmark snapshot query for large tables diff --git a/style/scalastyle.xml b/style/scalastyle.xml index 74d7b9d73a203..a1b4cdbb6dafa 100644 --- a/style/scalastyle.xml +++ b/style/scalastyle.xml @@ -27,7 +27,7 @@ - + @@ -113,7 +113,7 @@ - +