You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by yi...@apache.org on 2023/02/13 06:24:04 UTC

[hudi] branch release-0.13.0 updated (820006e025a -> fe664886029)

This is an automated email from the ASF dual-hosted git repository.

yihua pushed a change to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git


    from 820006e025a [HUDI-5718] Unsupported Operation Exception for compaction (#7874)
     new 847e7a975bf [HUDI-5758] Restoring state of `HoodieKey` to make sure it's binary compatible w/ its state in 0.12 (#7917)
     new 7ccf6e67827 [HUDI-5768] Fix Spark Datasource read of metadata table (#7924)
     new d4106f35b4a [HUDI-5764] Rollback delta commits from `HoodieIndexer` lazily in metadata table (#7921)
     new 4254fc9f482 [HUDI-5771] Improve deploy script of release artifacts (#7927)
     new fe664886029 Bumping release candidate number 3

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 docker/hoodie/hadoop/base/pom.xml                  |   2 +-
 docker/hoodie/hadoop/base_java11/pom.xml           |   2 +-
 docker/hoodie/hadoop/datanode/pom.xml              |   2 +-
 docker/hoodie/hadoop/historyserver/pom.xml         |   2 +-
 docker/hoodie/hadoop/hive_base/pom.xml             |   2 +-
 docker/hoodie/hadoop/namenode/pom.xml              |   2 +-
 docker/hoodie/hadoop/pom.xml                       |   2 +-
 docker/hoodie/hadoop/prestobase/pom.xml            |   2 +-
 docker/hoodie/hadoop/spark_base/pom.xml            |   2 +-
 docker/hoodie/hadoop/sparkadhoc/pom.xml            |   2 +-
 docker/hoodie/hadoop/sparkmaster/pom.xml           |   2 +-
 docker/hoodie/hadoop/sparkworker/pom.xml           |   2 +-
 docker/hoodie/hadoop/trinobase/pom.xml             |   2 +-
 docker/hoodie/hadoop/trinocoordinator/pom.xml      |   2 +-
 docker/hoodie/hadoop/trinoworker/pom.xml           |   2 +-
 hudi-aws/pom.xml                                   |   4 +-
 hudi-cli/pom.xml                                   |   2 +-
 hudi-client/hudi-client-common/pom.xml             |   4 +-
 .../hudi/client/BaseHoodieTableServiceClient.java  |  48 +++++++++
 .../apache/hudi/client/BaseHoodieWriteClient.java  |  13 +++
 .../metadata/HoodieBackedTableMetadataWriter.java  |  34 ++++---
 .../java/org/apache/hudi/table/HoodieTable.java    |  38 +++++++-
 .../table/action/index/RunIndexActionExecutor.java |   5 +-
 hudi-client/hudi-flink-client/pom.xml              |   4 +-
 .../FlinkHoodieBackedTableMetadataWriter.java      |  21 +++-
 .../org/apache/hudi/table/HoodieFlinkTable.java    |  12 ++-
 hudi-client/hudi-java-client/pom.xml               |   4 +-
 hudi-client/hudi-spark-client/pom.xml              |   4 +-
 .../SparkHoodieBackedTableMetadataWriter.java      |  20 +++-
 .../org/apache/hudi/table/HoodieSparkTable.java    |  10 +-
 .../apache/spark/HoodieSparkKryoRegistrar.scala    |  25 ++++-
 hudi-client/pom.xml                                |   2 +-
 hudi-common/pom.xml                                |   2 +-
 .../org/apache/hudi/common/model/DeleteRecord.java |   9 ++
 .../org/apache/hudi/common/model/HoodieKey.java    |  28 ++----
 .../common/table/log/block/HoodieDeleteBlock.java  |   2 +
 .../hudi/metadata/HoodieBackedTableMetadata.java   |  12 ++-
 .../hudi/metadata/HoodieTableMetadataUtil.java     |  20 ++++
 hudi-examples/hudi-examples-common/pom.xml         |   2 +-
 hudi-examples/hudi-examples-flink/pom.xml          |   2 +-
 hudi-examples/hudi-examples-java/pom.xml           |   2 +-
 hudi-examples/hudi-examples-spark/pom.xml          |   2 +-
 hudi-examples/pom.xml                              |   2 +-
 hudi-flink-datasource/hudi-flink/pom.xml           |   4 +-
 hudi-flink-datasource/hudi-flink1.13.x/pom.xml     |   4 +-
 hudi-flink-datasource/hudi-flink1.14.x/pom.xml     |   4 +-
 hudi-flink-datasource/hudi-flink1.15.x/pom.xml     |   4 +-
 hudi-flink-datasource/hudi-flink1.16.x/pom.xml     |   4 +-
 hudi-flink-datasource/pom.xml                      |   4 +-
 hudi-gcp/pom.xml                                   |   2 +-
 hudi-hadoop-mr/pom.xml                             |   2 +-
 hudi-integ-test/pom.xml                            |   2 +-
 hudi-kafka-connect/pom.xml                         |   4 +-
 .../hudi-metaserver/hudi-metaserver-client/pom.xml |   2 +-
 .../hudi-metaserver/hudi-metaserver-server/pom.xml |   2 +-
 hudi-platform-service/hudi-metaserver/pom.xml      |   4 +-
 hudi-platform-service/pom.xml                      |   2 +-
 hudi-spark-datasource/hudi-spark-common/pom.xml    |   4 +-
 .../scala/org/apache/hudi/HoodieBaseRelation.scala |   5 +-
 hudi-spark-datasource/hudi-spark/pom.xml           |   4 +-
 hudi-spark-datasource/hudi-spark2-common/pom.xml   |   2 +-
 hudi-spark-datasource/hudi-spark2/pom.xml          |   4 +-
 hudi-spark-datasource/hudi-spark3-common/pom.xml   |   2 +-
 hudi-spark-datasource/hudi-spark3.1.x/pom.xml      |   4 +-
 hudi-spark-datasource/hudi-spark3.2.x/pom.xml      |   4 +-
 .../hudi-spark3.2plus-common/pom.xml               |   2 +-
 hudi-spark-datasource/hudi-spark3.3.x/pom.xml      |   4 +-
 hudi-spark-datasource/pom.xml                      |   2 +-
 hudi-sync/hudi-adb-sync/pom.xml                    |   2 +-
 hudi-sync/hudi-datahub-sync/pom.xml                |   2 +-
 hudi-sync/hudi-hive-sync/pom.xml                   |   2 +-
 hudi-sync/hudi-sync-common/pom.xml                 |   2 +-
 hudi-sync/pom.xml                                  |   2 +-
 hudi-tests-common/pom.xml                          |   2 +-
 hudi-timeline-service/pom.xml                      |   2 +-
 hudi-utilities/pom.xml                             |   2 +-
 .../apache/hudi/utilities/TestHoodieIndexer.java   | 108 +++++++++++++++++++--
 packaging/hudi-aws-bundle/pom.xml                  |   2 +-
 packaging/hudi-cli-bundle/pom.xml                  |   2 +-
 packaging/hudi-datahub-sync-bundle/pom.xml         |   2 +-
 packaging/hudi-flink-bundle/pom.xml                |   2 +-
 packaging/hudi-gcp-bundle/pom.xml                  |   2 +-
 packaging/hudi-hadoop-mr-bundle/pom.xml            |   2 +-
 packaging/hudi-hive-sync-bundle/pom.xml            |   2 +-
 packaging/hudi-integ-test-bundle/pom.xml           |   2 +-
 packaging/hudi-kafka-connect-bundle/pom.xml        |   2 +-
 packaging/hudi-metaserver-server-bundle/pom.xml    |   2 +-
 packaging/hudi-presto-bundle/pom.xml               |   2 +-
 packaging/hudi-spark-bundle/pom.xml                |   2 +-
 packaging/hudi-timeline-server-bundle/pom.xml      |   2 +-
 packaging/hudi-trino-bundle/pom.xml                |   2 +-
 packaging/hudi-utilities-bundle/pom.xml            |   2 +-
 packaging/hudi-utilities-slim-bundle/pom.xml       |   2 +-
 pom.xml                                            |   2 +-
 scripts/release/deploy_staging_jars.sh             |  74 +++++++-------
 95 files changed, 475 insertions(+), 201 deletions(-)


[hudi] 04/05: [HUDI-5771] Improve deploy script of release artifacts (#7927)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 4254fc9f4829733c24d4c22c78ae855df7755798
Author: Y Ethan Guo <et...@gmail.com>
AuthorDate: Sun Feb 12 22:14:31 2023 -0800

    [HUDI-5771] Improve deploy script of release artifacts (#7927)
    
    The current scripts/release/deploy_staging_jars.sh took around 6 hours to upload all release artifacts to the Apache Nexus staging repository, which is too long.  This commit cuts down the upload time by 70% to <2 hours, without changing the intended jars for uploads.
---
 scripts/release/deploy_staging_jars.sh | 74 ++++++++++++++++------------------
 1 file changed, 34 insertions(+), 40 deletions(-)

diff --git a/scripts/release/deploy_staging_jars.sh b/scripts/release/deploy_staging_jars.sh
index 049e5ee7144..7d44e5ffa96 100755
--- a/scripts/release/deploy_staging_jars.sh
+++ b/scripts/release/deploy_staging_jars.sh
@@ -36,38 +36,41 @@ if [ "$#" -gt "1" ]; then
   exit 1
 fi
 
-BUNDLE_MODULES=$(find -s packaging -name 'hudi-*-bundle' -type d)
-BUNDLE_MODULES_EXCLUDED="-${BUNDLE_MODULES//$'\n'/,-}"
-
 declare -a ALL_VERSION_OPTS=(
-# upload all module jars and bundle jars
-"-Dscala-2.11 -Dspark2.4 -pl $BUNDLE_MODULES_EXCLUDED"
-"-Dscala-2.12 -Dspark2.4 -pl $BUNDLE_MODULES_EXCLUDED"
-"-Dscala-2.12 -Dspark3.3 -pl $BUNDLE_MODULES_EXCLUDED"
-"-Dscala-2.12 -Dspark3.2 -pl $BUNDLE_MODULES_EXCLUDED"
-"-Dscala-2.12 -Dspark3.1"  # this profile goes last in this section to ensure bundles use avro 1.8
-
-# spark bundles
-"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am"
+# Upload Spark specific modules and bundle jars
+# For Spark 2.4, Scala 2.11:
+# hudi-spark-common_2.11
+# hudi-spark_2.11
+# hudi-spark2_2.11
+# hudi-utilities_2.11
+# hudi-cli-bundle_2.11
+# hudi-spark2.4-bundle_2.11
+# hudi-utilities-bundle_2.11
+# hudi-utilities-slim-bundle_2.11
+"-Dscala-2.11 -Dspark2.4 -pl hudi-spark-datasource/hudi-spark-common,hudi-spark-datasource/hudi-spark2,hudi-spark-datasource/hudi-spark,hudi-utilities,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle,packaging/hudi-utilities-bundle,packaging/hudi-utilities-slim-bundle -am"
+# For Spark 2.4, Scala 2.12:
+# hudi-spark2.4-bundle_2.12
 "-Dscala-2.12 -Dspark2.4 -pl packaging/hudi-spark-bundle -am"
-"-Dscala-2.12 -Dspark3.3 -pl packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am"
-"-Dscala-2.12 -Dspark3.2 -pl packaging/hudi-spark-bundle -am"
-"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-spark-bundle -am"
-
-# spark bundles (legacy) (not overwriting previous uploads as these jar names are unique)
+# For Spark 3.2, Scala 2.12:
+# hudi-spark3.2.x_2.12
+# hudi-spark3.2plus-common
+# hudi-spark3.2-bundle_2.12
+"-Dscala-2.12 -Dspark3.2 -pl hudi-spark-datasource/hudi-spark3.2.x,hudi-spark-datasource/hudi-spark3.2plus-common,packaging/hudi-spark-bundle -am"
+# For Spark 3.1, Scala 2.12:
+# All other modules and bundles using avro 1.8
+"-Dscala-2.12 -Dspark3.1"
+# For Spark 3.3, Scala 2.12:
+# hudi-spark3.3.x_2.12
+# hudi-cli-bundle_2.12
+# hudi-spark3.3-bundle_2.12
+"-Dscala-2.12 -Dspark3.3 -pl hudi-spark-datasource/hudi-spark3.3.x,packaging/hudi-spark-bundle,packaging/hudi-cli-bundle -am"
+
+# Upload legacy Spark bundles (not overwriting previous uploads as these jar names are unique)
 "-Dscala-2.11 -Dspark2 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark-bundle_2.11
 "-Dscala-2.12 -Dspark2 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark-bundle_2.12
 "-Dscala-2.12 -Dspark3 -pl packaging/hudi-spark-bundle -am" # for legacy bundle name hudi-spark3-bundle_2.12
 
-# utilities bundles (legacy) (overwriting previous uploads)
-"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-utilities-bundle -am" # hudi-utilities-bundle_2.11 is for spark 2.4 only
-"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-utilities-bundle -am" # hudi-utilities-bundle_2.12 is for spark 3.1 only
-
-# utilities slim bundles
-"-Dscala-2.11 -Dspark2.4 -pl packaging/hudi-utilities-slim-bundle -am" # hudi-utilities-slim-bundle_2.11
-"-Dscala-2.12 -Dspark3.1 -pl packaging/hudi-utilities-slim-bundle -am" # hudi-utilities-slim-bundle_2.12
-
-# flink bundles (overwriting previous uploads)
+# Upload Flink bundles (overwriting previous uploads)
 "-Dscala-2.12 -Dflink1.13 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am"
 "-Dscala-2.12 -Dflink1.14 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am"
 "-Dscala-2.12 -Dflink1.15 -Davro.version=1.10.0 -pl packaging/hudi-flink-bundle -am"
@@ -105,20 +108,11 @@ COMMON_OPTIONS="-DdeployArtifacts=true -DskipTests -DretryFailedDeploymentCount=
 for v in "${ALL_VERSION_OPTS[@]}"
 do
   # TODO: consider cleaning all modules by listing directories instead of specifying profile
-  if [[ "$v" == *"$BUNDLE_MODULES_EXCLUDED" ]]; then
-    # When deploying jars with bundle exclusions, we still need to build the bundles,
-    # by removing "-pl -packaging/hudi-aws-bundle...", otherwise the build fails.
-    v1=${v%${BUNDLE_MODULES_EXCLUDED}}
-    echo "Cleaning everything before any deployment"
-    $MVN clean $COMMON_OPTIONS ${v1%-pl }
-    echo "Building with options ${v1%-pl }"
-    $MVN install $COMMON_OPTIONS ${v1%-pl }
-  else
-    echo "Cleaning everything before any deployment"
-    $MVN clean $COMMON_OPTIONS ${v}
-    echo "Building with options ${v}"
-    $MVN install $COMMON_OPTIONS ${v}
-  fi
+  echo "Cleaning everything before any deployment"
+  $MVN clean $COMMON_OPTIONS ${v}
+  echo "Building with options ${v}"
+  $MVN install $COMMON_OPTIONS ${v}
+
   echo "Deploying to repository.apache.org with version options ${v%-am}"
   # remove `-am` option to only deploy intended modules
   $MVN deploy $COMMON_OPTIONS ${v%-am}


[hudi] 03/05: [HUDI-5764] Rollback delta commits from `HoodieIndexer` lazily in metadata table (#7921)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit d4106f35b4aee53ea5cb1430288f397b37c81183
Author: Y Ethan Guo <et...@gmail.com>
AuthorDate: Sun Feb 12 03:30:10 2023 -0800

    [HUDI-5764] Rollback delta commits from `HoodieIndexer` lazily in metadata table (#7921)
    
    Fixes two issues:
    - Makes the rollback of indexing delta commit lazy in the metadata table, otherwise, it would be cleaned up eagerly by other regular writes.
    - Uses a suffix (004) appending to the up-to-instant used by the async index to avoid collision with existing completed delta commit of the same instant time.
---
 .../hudi/client/BaseHoodieTableServiceClient.java  |  48 +++++++++
 .../apache/hudi/client/BaseHoodieWriteClient.java  |  13 +++
 .../metadata/HoodieBackedTableMetadataWriter.java  |  34 ++++---
 .../java/org/apache/hudi/table/HoodieTable.java    |  38 +++++++-
 .../table/action/index/RunIndexActionExecutor.java |   5 +-
 .../FlinkHoodieBackedTableMetadataWriter.java      |  21 +++-
 .../org/apache/hudi/table/HoodieFlinkTable.java    |  12 ++-
 .../SparkHoodieBackedTableMetadataWriter.java      |  20 +++-
 .../org/apache/hudi/table/HoodieSparkTable.java    |  10 +-
 .../hudi/metadata/HoodieBackedTableMetadata.java   |  12 ++-
 .../hudi/metadata/HoodieTableMetadataUtil.java     |  20 ++++
 .../apache/hudi/utilities/TestHoodieIndexer.java   | 108 +++++++++++++++++++--
 12 files changed, 298 insertions(+), 43 deletions(-)

diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
index 390bc4b9714..301ed61bf4e 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java
@@ -48,6 +48,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieRollbackException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
 import org.apache.hudi.metadata.HoodieTableMetadataWriter;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -71,6 +72,7 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
+import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit;
 
 public abstract class BaseHoodieTableServiceClient<O> extends BaseHoodieClient implements RunsTableService {
 
@@ -659,8 +661,41 @@ public abstract class BaseHoodieTableServiceClient<O> extends BaseHoodieClient i
     return infoMap;
   }
 
+  /**
+   * Rolls back the failed delta commits corresponding to the indexing action.
+   * Such delta commits are identified based on the suffix `METADATA_INDEXER_TIME_SUFFIX` ("004").
+   * <p>
+   * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks
+   *  in the metadata table is landed.
+   *
+   * @return {@code true} if rollback happens; {@code false} otherwise.
+   */
+  protected boolean rollbackFailedIndexingCommits() {
+    HoodieTable table = createTable(config, hadoopConf);
+    List<String> instantsToRollback = getFailedIndexingCommitsToRollback(table.getMetaClient());
+    Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(table.getMetaClient());
+    instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
+    rollbackFailedWrites(pendingRollbacks);
+    return !pendingRollbacks.isEmpty();
+  }
+
+  protected List<String> getFailedIndexingCommitsToRollback(HoodieTableMetaClient metaClient) {
+    Stream<HoodieInstant> inflightInstantsStream = metaClient.getCommitsTimeline()
+        .filter(instant -> !instant.isCompleted()
+            && isIndexingCommit(instant.getTimestamp()))
+        .getInstantsAsStream();
+    return inflightInstantsStream.filter(instant -> {
+      try {
+        return heartbeatClient.isHeartbeatExpired(instant.getTimestamp());
+      } catch (IOException io) {
+        throw new HoodieException("Failed to check heartbeat for instant " + instant, io);
+      }
+    }).map(HoodieInstant::getTimestamp).collect(Collectors.toList());
+  }
+
   /**
    * Rollback all failed writes.
+   *
    * @return true if rollback was triggered. false otherwise.
    */
   protected Boolean rollbackFailedWrites() {
@@ -699,6 +734,19 @@ public abstract class BaseHoodieTableServiceClient<O> extends BaseHoodieClient i
     Stream<HoodieInstant> inflightInstantsStream = getInflightTimelineExcludeCompactionAndClustering(metaClient)
         .getReverseOrderedInstants();
     if (cleaningPolicy.isEager()) {
+      // Metadata table uses eager cleaning policy, but we need to exclude inflight delta commits
+      // from the async indexer (`HoodieIndexer`).
+      // TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks in the
+      //  metadata table is landed.
+      if (HoodieTableMetadata.isMetadataTable(metaClient.getBasePathV2().toString())) {
+        return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> {
+          if (curInstantTime.isPresent()) {
+            return !entry.equals(curInstantTime.get());
+          } else {
+            return !isIndexingCommit(entry);
+          }
+        }).collect(Collectors.toList());
+      }
       return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> {
         if (curInstantTime.isPresent()) {
           return !entry.equals(curInstantTime.get());
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
index 17956479762..10c0db97151 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
@@ -1279,8 +1279,21 @@ public abstract class BaseHoodieWriteClient<T, I, K, O> extends BaseHoodieClient
     }
   }
 
+  /**
+   * Rolls back the failed delta commits corresponding to the indexing action.
+   * <p>
+   * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks
+   *  in the metadata table is landed.
+   *
+   * @return {@code true} if rollback happens; {@code false} otherwise.
+   */
+  public boolean lazyRollbackFailedIndexing() {
+    return tableServiceClient.rollbackFailedIndexingCommits();
+  }
+
   /**
    * Rollback failed writes if any.
+   *
    * @return true if rollback happened. false otherwise.
    */
   public boolean rollbackFailedWrites() {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
index 5e8367e2095..3338872efbb 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
@@ -93,6 +93,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER;
 import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
 import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED;
 import static org.apache.hudi.common.table.timeline.HoodieTimeline.getIndexInflightInstant;
@@ -100,6 +101,7 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deseri
 import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING;
 import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
 import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
+import static org.apache.hudi.metadata.HoodieTableMetadataUtil.METADATA_INDEXER_TIME_SUFFIX;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions;
 
@@ -134,15 +136,17 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
   /**
    * Hudi backed table metadata writer.
    *
-   * @param hadoopConf               - Hadoop configuration to use for the metadata writer
-   * @param writeConfig              - Writer config
-   * @param engineContext            - Engine context
-   * @param actionMetadata           - Optional action metadata to help decide initialize operations
-   * @param <T>                      - Action metadata types extending Avro generated SpecificRecordBase
-   * @param inflightInstantTimestamp - Timestamp of any instant in progress
+   * @param hadoopConf                 Hadoop configuration to use for the metadata writer
+   * @param writeConfig                Writer config
+   * @param failedWritesCleaningPolicy Cleaning policy on failed writes
+   * @param engineContext              Engine context
+   * @param actionMetadata             Optional action metadata to help decide initialize operations
+   * @param <T>                        Action metadata types extending Avro generated SpecificRecordBase
+   * @param inflightInstantTimestamp   Timestamp of any instant in progress
    */
   protected <T extends SpecificRecordBase> HoodieBackedTableMetadataWriter(Configuration hadoopConf,
                                                                            HoodieWriteConfig writeConfig,
+                                                                           HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
                                                                            HoodieEngineContext engineContext,
                                                                            Option<T> actionMetadata,
                                                                            Option<String> inflightInstantTimestamp) {
@@ -154,7 +158,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
 
     if (writeConfig.isMetadataTableEnabled()) {
       this.tableName = writeConfig.getTableName() + METADATA_TABLE_NAME_SUFFIX;
-      this.metadataWriteConfig = createMetadataWriteConfig(writeConfig);
+      this.metadataWriteConfig = createMetadataWriteConfig(writeConfig, failedWritesCleaningPolicy);
       enabled = true;
 
       // Inline compaction and auto clean is required as we dont expose this table outside
@@ -181,7 +185,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
 
   public HoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig,
                                          HoodieEngineContext engineContext) {
-    this(hadoopConf, writeConfig, engineContext, Option.empty(), Option.empty());
+    this(hadoopConf, writeConfig, EAGER, engineContext, Option.empty(), Option.empty());
   }
 
   /**
@@ -232,11 +236,14 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
   protected abstract void initRegistry();
 
   /**
-   * Create a {@code HoodieWriteConfig} to use for the Metadata Table.
+   * Create a {@code HoodieWriteConfig} to use for the Metadata Table.  This is used by async
+   * indexer only.
    *
-   * @param writeConfig {@code HoodieWriteConfig} of the main dataset writer
+   * @param writeConfig                {@code HoodieWriteConfig} of the main dataset writer
+   * @param failedWritesCleaningPolicy Cleaning policy on failed writes
    */
-  private HoodieWriteConfig createMetadataWriteConfig(HoodieWriteConfig writeConfig) {
+  private HoodieWriteConfig createMetadataWriteConfig(
+      HoodieWriteConfig writeConfig, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy) {
     int parallelism = writeConfig.getMetadataInsertParallelism();
 
     int minCommitsToKeep = Math.max(writeConfig.getMetadataMinCommitsToKeep(), writeConfig.getMinCommitsToKeep());
@@ -268,7 +275,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
             .withAutoClean(false)
             .withCleanerParallelism(parallelism)
             .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)
-            .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
+            .withFailedWritesCleaningPolicy(failedWritesCleaningPolicy)
             .retainCommits(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED.defaultValue())
             .build())
         // we will trigger archive manually, to ensure only regular writer invokes it
@@ -875,7 +882,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
     inflightIndexes.addAll(indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()));
     dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightIndexes));
     HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps());
-    initialCommit(indexUptoInstantTime, partitionTypes);
+    initialCommit(indexUptoInstantTime + METADATA_INDEXER_TIME_SUFFIX, partitionTypes);
   }
 
   /**
@@ -1069,6 +1076,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
     // delta commits synced over will not have an instant time lesser than the last completed instant on the
     // metadata table.
     writeClient.clean(instantTime + "002");
+    writeClient.lazyRollbackFailedIndexing();
   }
 
   /**
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 591ebc430dc..4102515ae01 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
@@ -41,6 +41,7 @@ import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.fs.FailSafeConsistencyGuard;
 import org.apache.hudi.common.fs.OptimisticConsistencyGuard;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -100,6 +101,8 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import static org.apache.hudi.avro.AvroSchemaUtils.isSchemaCompatible;
+import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER;
+import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.LAZY;
 import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS;
 import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition;
@@ -872,7 +875,8 @@ public abstract class HoodieTable<T, I, K, O> implements Serializable {
    * @return instance of {@link HoodieTableMetadataWriter}
    */
   public final Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp) {
-    return getMetadataWriter(triggeringInstantTimestamp, Option.empty());
+    return getMetadataWriter(
+        triggeringInstantTimestamp, EAGER, Option.empty());
   }
 
   /**
@@ -895,6 +899,29 @@ public abstract class HoodieTable<T, I, K, O> implements Serializable {
     }
   }
 
+  /**
+   * Gets the metadata writer for async indexer.
+   *
+   * @param triggeringInstantTimestamp The instant that is triggering this metadata write.
+   * @return An instance of {@link HoodieTableMetadataWriter}.
+   */
+  public Option<HoodieTableMetadataWriter> getIndexingMetadataWriter(String triggeringInstantTimestamp) {
+    return getMetadataWriter(triggeringInstantTimestamp, LAZY, Option.empty());
+  }
+
+  /**
+   * Gets the metadata writer for regular writes.
+   *
+   * @param triggeringInstantTimestamp The instant that is triggering this metadata write.
+   * @param actionMetadata             Optional action metadata.
+   * @param <R>                        Action metadata type.
+   * @return An instance of {@link HoodieTableMetadataWriter}.
+   */
+  public <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(
+      String triggeringInstantTimestamp, Option<R> actionMetadata) {
+    return getMetadataWriter(triggeringInstantTimestamp, EAGER, actionMetadata);
+  }
+
   /**
    * Get Table metadata writer.
    * <p>
@@ -905,11 +932,14 @@ public abstract class HoodieTable<T, I, K, O> implements Serializable {
    * are blocked from doing the similar initial metadata table creation and
    * the bootstrapping.
    *
-   * @param triggeringInstantTimestamp - The instant that is triggering this metadata write
+   * @param triggeringInstantTimestamp The instant that is triggering this metadata write
+   * @param failedWritesCleaningPolicy Cleaning policy on failed writes
    * @return instance of {@link HoodieTableMetadataWriter}
    */
-  public <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
-                                                                                            Option<R> actionMetadata) {
+  protected <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(
+      String triggeringInstantTimestamp,
+      HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
+      Option<R> actionMetadata) {
     // Each engine is expected to override this and
     // provide the actual metadata writer, if enabled.
     return Option.empty();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java
index 2fcbfb2b2e5..19aab3629d5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java
@@ -138,8 +138,9 @@ public class RunIndexActionExecutor<T, I, K, O> extends BaseActionExecutor<T, I,
       List<HoodieIndexPartitionInfo> finalIndexPartitionInfos = null;
       if (!firstTimeInitializingMetadataTable) {
         // start indexing for each partition
-        HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime)
-            .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime)));
+        HoodieTableMetadataWriter metadataWriter = table.getIndexingMetadataWriter(instantTime)
+            .orElseThrow(() -> new HoodieIndexException(String.format(
+                "Could not get metadata writer to run index action for instant: %s", instantTime)));
         // this will only build index upto base instant as generated by the plan, we will be doing catchup later
         String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant();
         LOG.info("Starting Index Building with base instant: " + indexUptoInstant);
diff --git a/hudi-client/hudi-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 aa70f5835c8..cd45685e13e 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java
@@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -42,6 +43,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER;
+
 /**
  * Flink hoodie backed table metadata writer.
  */
@@ -58,23 +61,35 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
                                                                                 HoodieWriteConfig writeConfig,
                                                                                 HoodieEngineContext context,
                                                                                 Option<T> actionMetadata) {
-    return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, Option.empty());
+    return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, EAGER, context, actionMetadata, Option.empty());
+  }
+
+  public static <T extends SpecificRecordBase> HoodieTableMetadataWriter create(Configuration conf,
+                                                                                HoodieWriteConfig writeConfig,
+                                                                                HoodieEngineContext context,
+                                                                                Option<T> actionMetadata,
+                                                                                Option<String> inFlightInstantTimestamp) {
+    return new FlinkHoodieBackedTableMetadataWriter(
+        conf, writeConfig, EAGER, context, actionMetadata, inFlightInstantTimestamp);
   }
 
   public static <T extends SpecificRecordBase> HoodieTableMetadataWriter create(Configuration conf,
                                                                                 HoodieWriteConfig writeConfig,
+                                                                                HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
                                                                                 HoodieEngineContext context,
                                                                                 Option<T> actionMetadata,
                                                                                 Option<String> inFlightInstantTimestamp) {
-    return new FlinkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata, inFlightInstantTimestamp);
+    return new FlinkHoodieBackedTableMetadataWriter(
+        conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inFlightInstantTimestamp);
   }
 
   <T extends SpecificRecordBase> FlinkHoodieBackedTableMetadataWriter(Configuration hadoopConf,
                                                                       HoodieWriteConfig writeConfig,
+                                                                      HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
                                                                       HoodieEngineContext engineContext,
                                                                       Option<T> actionMetadata,
                                                                       Option<String> inFlightInstantTimestamp) {
-    super(hadoopConf, writeConfig, engineContext, actionMetadata, inFlightInstantTimestamp);
+    super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inFlightInstantTimestamp);
   }
 
   @Override
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
index 3d77844df6f..422fe310b0c 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java
@@ -22,6 +22,7 @@ import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.common.HoodieFlinkEngineContext;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -98,11 +99,14 @@ public abstract class HoodieFlinkTable<T>
    * @return instance of {@link HoodieTableMetadataWriter}
    */
   @Override
-  public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
-                                                                                            Option<T> actionMetadata) {
+  protected <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(
+      String triggeringInstantTimestamp,
+      HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
+      Option<T> actionMetadata) {
     if (config.isMetadataTableEnabled()) {
-      return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config,
-          context, actionMetadata, Option.of(triggeringInstantTimestamp)));
+      return Option.of(FlinkHoodieBackedTableMetadataWriter.create(
+          context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context,
+          actionMetadata, Option.of(triggeringInstantTimestamp)));
     } else {
       return Option.empty();
     }
diff --git a/hudi-client/hudi-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 23537f6f798..37222c8266a 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
@@ -24,6 +24,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.metrics.Registry;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -49,6 +50,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+import static org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy.EAGER;
+
 public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
 
   private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class);
@@ -73,10 +76,20 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
                                                                                 HoodieEngineContext context,
                                                                                 Option<T> actionMetadata,
                                                                                 Option<String> inflightInstantTimestamp) {
-    return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata,
-                                                    inflightInstantTimestamp);
+    return new SparkHoodieBackedTableMetadataWriter(
+        conf, writeConfig, EAGER, context, actionMetadata, inflightInstantTimestamp);
   }
 
+  public static <T extends SpecificRecordBase> HoodieTableMetadataWriter create(Configuration conf,
+                                                                                HoodieWriteConfig writeConfig,
+                                                                                HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
+                                                                                HoodieEngineContext context,
+                                                                                Option<T> actionMetadata,
+                                                                                Option<String> inflightInstantTimestamp) {
+    return new SparkHoodieBackedTableMetadataWriter(
+        conf, writeConfig, failedWritesCleaningPolicy, context, actionMetadata, inflightInstantTimestamp);
+  }
+  
   public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig,
                                                  HoodieEngineContext context) {
     return create(conf, writeConfig, context, Option.empty(), Option.empty());
@@ -84,10 +97,11 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
 
   <T extends SpecificRecordBase> SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf,
                                                                       HoodieWriteConfig writeConfig,
+                                                                      HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
                                                                       HoodieEngineContext engineContext,
                                                                       Option<T> actionMetadata,
                                                                       Option<String> inflightInstantTimestamp) {
-    super(hadoopConf, writeConfig, engineContext, actionMetadata, inflightInstantTimestamp);
+    super(hadoopConf, writeConfig, failedWritesCleaningPolicy, engineContext, actionMetadata, inflightInstantTimestamp);
   }
 
   @Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java
index 112deccf8df..d6796a7a4d4 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java
@@ -21,6 +21,7 @@ package org.apache.hudi.table;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -89,14 +90,17 @@ public abstract class HoodieSparkTable<T>
    * @return instance of {@link HoodieTableMetadataWriter}
    */
   @Override
-  public <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
-                                                                                            Option<R> actionMetadata) {
+  protected <R extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(
+      String triggeringInstantTimestamp,
+      HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy,
+      Option<R> actionMetadata) {
     if (config.isMetadataTableEnabled()) {
       // Create the metadata table writer. First time after the upgrade this creation might trigger
       // metadata table bootstrapping. Bootstrapping process could fail and checking the table
       // existence after the creation is needed.
       final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(
-          context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp));
+          context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context,
+          actionMetadata, Option.of(triggeringInstantTimestamp));
       // even with metadata enabled, some index could have been disabled
       // delete metadata partitions corresponding to such indexes
       deleteMetadataIndexIfNecessary();
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
index 1ccc14176a1..f3538127955 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
@@ -53,9 +53,9 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieMetadataException;
 import org.apache.hudi.exception.TableNotFoundException;
-import org.apache.hudi.util.Transient;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.io.storage.HoodieSeekingFileReader;
+import org.apache.hudi.util.Transient;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
@@ -83,6 +83,7 @@ import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_BL
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_FILES;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getFileSystemView;
+import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit;
 
 /**
  * Table metadata provided by an internal DFS backed Hudi metadata table.
@@ -482,6 +483,15 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     Set<String> validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstantsAsStream()
         .map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
 
+    // We should also add completed indexing delta commits in the metadata table, as they do not
+    // have corresponding completed instant in the data table
+    validInstantTimestamps.addAll(
+        metadataMetaClient.getActiveTimeline()
+            .filter(instant -> instant.isCompleted() && isIndexingCommit(instant.getTimestamp()))
+            .getInstants().stream()
+            .map(HoodieInstant::getTimestamp)
+            .collect(Collectors.toList()));
+
     // For any rollbacks and restores, we cannot neglect the instants that they are rolling back.
     // The rollback instant should be more recent than the start of the timeline for it to have rolled back any
     // instant which we have a log block for.
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 d37dbab3d82..81ba4f2a66b 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
@@ -94,6 +94,7 @@ import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
 import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields;
 import static org.apache.hudi.avro.HoodieAvroUtils.convertValueForSpecificDataTypes;
 import static org.apache.hudi.avro.HoodieAvroUtils.getNestedFieldSchemaFromWriteSchema;
+import static org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.MILLIS_INSTANT_ID_LENGTH;
 import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
 import static org.apache.hudi.common.util.ValidationUtils.checkState;
 import static org.apache.hudi.metadata.HoodieMetadataPayload.unwrapStatisticValueWrapper;
@@ -111,6 +112,11 @@ public class HoodieTableMetadataUtil {
   public static final String PARTITION_NAME_COLUMN_STATS = "column_stats";
   public static final String PARTITION_NAME_BLOOM_FILTERS = "bloom_filters";
 
+  // This suffix used by the delta commits from async indexer (`HoodieIndexer`),
+  // when the `indexUptoInstantTime` already exists in the metadata table,
+  // to avoid collision.
+  public static final String METADATA_INDEXER_TIME_SUFFIX = "004";
+
   /**
    * Returns whether the files partition of metadata table is ready for read.
    *
@@ -1380,4 +1386,18 @@ public class HoodieTableMetadataUtil {
     inflightAndCompletedPartitions.addAll(tableConfig.getMetadataPartitions());
     return inflightAndCompletedPartitions;
   }
+
+  /**
+   * Checks if a delta commit in metadata table is written by async indexer.
+   * <p>
+   * TODO(HUDI-5733): This should be cleaned up once the proper fix of rollbacks in the
+   *  metadata table is landed.
+   *
+   * @param instantTime Instant time to check.
+   * @return {@code true} if from async indexer; {@code false} otherwise.
+   */
+  public static boolean isIndexingCommit(String instantTime) {
+    return instantTime.length() == MILLIS_INSTANT_ID_LENGTH + METADATA_INDEXER_TIME_SUFFIX.length()
+        && instantTime.endsWith(METADATA_INDEXER_TIME_SUFFIX);
+  }
 }
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java
index ac7b86f4cfa..67504cb957d 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java
@@ -21,18 +21,25 @@ package org.apache.hudi.utilities;
 
 import org.apache.hudi.avro.model.HoodieIndexCommitMetadata;
 import org.apache.hudi.avro.model.HoodieIndexPartitionInfo;
+import org.apache.hudi.avro.model.HoodieIndexPlan;
+import org.apache.hudi.avro.model.HoodieRollbackMetadata;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.heartbeat.HoodieHeartbeatClient;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
 import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.metadata.HoodieBackedTableMetadata;
 import org.apache.hudi.metadata.HoodieTableMetadataUtil;
 import org.apache.hudi.metadata.MetadataPartitionType;
 import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
@@ -56,6 +63,9 @@ import java.util.stream.Stream;
 
 import static org.apache.hudi.common.table.HoodieTableMetaClient.reload;
 import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED;
+import static org.apache.hudi.config.HoodieWriteConfig.CLIENT_HEARTBEAT_INTERVAL_IN_MS;
+import static org.apache.hudi.config.HoodieWriteConfig.CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES;
+import static org.apache.hudi.metadata.HoodieTableMetadataUtil.METADATA_INDEXER_TIME_SUFFIX;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getFileSystemView;
 import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists;
 import static org.apache.hudi.metadata.MetadataPartitionType.BLOOM_FILTERS;
@@ -123,7 +133,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     String tableName = "indexer_test";
     // enable files and bloom_filters on the regular write client
     HoodieMetadataConfig.Builder metadataConfigBuilder = getMetadataConfigBuilder(true, false).withMetadataIndexBloomFilter(true);
-    initializeWriteClient(metadataConfigBuilder.build(), tableName);
+    upsertToTable(metadataConfigBuilder.build(), tableName);
 
     // validate table config
     assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
@@ -138,7 +148,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     String tableName = "indexer_test";
     // enable files and bloom_filters on the regular write client
     HoodieMetadataConfig.Builder metadataConfigBuilder = getMetadataConfigBuilder(false, false).withMetadataIndexBloomFilter(true);
-    initializeWriteClient(metadataConfigBuilder.build(), tableName);
+    upsertToTable(metadataConfigBuilder.build(), tableName);
 
     // validate table config
     assertFalse(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
@@ -147,6 +157,78 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     indexMetadataPartitionsAndAssert(FILES, Collections.emptyList(), Arrays.asList(new MetadataPartitionType[] {COLUMN_STATS, BLOOM_FILTERS}), tableName);
   }
 
+  @Test
+  public void testIndexerWithWriter() throws IOException {
+    // Test the case where the indexer is running, i.e., the delta commit in the metadata table
+    // is inflight, while the regular writer is updating metadata table.
+    // The delta commit from the indexer should not be rolled back.
+    String tableName = "indexer_with_writer";
+    // Enable files and bloom_filters on the regular write client
+    HoodieMetadataConfig.Builder metadataConfigBuilder =
+        getMetadataConfigBuilder(true, false).withMetadataIndexBloomFilter(true);
+    HoodieMetadataConfig metadataConfig = metadataConfigBuilder.build();
+    upsertToTable(metadataConfig, tableName);
+
+    // Validate table config
+    assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
+    assertTrue(reload(metaClient).getTableConfig().getMetadataPartitions().contains(BLOOM_FILTERS.getPartitionPath()));
+
+    // Run async indexer, creating a new indexing instant in the data table and a new delta commit
+    // in the metadata table, with the suffix "004"
+    scheduleAndExecuteIndexing(COLUMN_STATS, tableName);
+
+    HoodieInstant indexingInstant = metaClient.getActiveTimeline()
+        .filter(i -> HoodieTimeline.INDEXING_ACTION.equals(i.getAction()))
+        .getInstants().get(0);
+    HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(
+        metaClient.getActiveTimeline().readIndexPlanAsBytes(indexingInstant).get());
+    String indexUptoInstantTime = indexPlan.getIndexPartitionInfos().get(0).getIndexUptoInstant();
+    HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata(
+        context(), metadataConfig, metaClient.getBasePathV2().toString(),
+        getWriteConfigBuilder(basePath(), tableName).build().getSpillableMapBasePath());
+    HoodieTableMetaClient metadataMetaClient = metadata.getMetadataMetaClient();
+    String mdtCommitTime = indexUptoInstantTime + METADATA_INDEXER_TIME_SUFFIX;
+    assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtCommitTime));
+
+    // Reverts both instants to inflight state, to simulate inflight indexing instants
+    metaClient.getActiveTimeline().revertToInflight(indexingInstant);
+    metaClient = reload(metaClient);
+
+    HoodieInstant mdtIndexingCommit = metadataMetaClient.getActiveTimeline()
+        .filter(i -> i.getTimestamp().equals(mdtCommitTime))
+        .getInstants().get(0);
+    metadataMetaClient.getActiveTimeline().revertToInflight(mdtIndexingCommit);
+    metadataMetaClient = reload(metadataMetaClient);
+    // Simulate heartbeats for ongoing write from async indexer in the metadata table
+    HoodieHeartbeatClient heartbeatClient = new HoodieHeartbeatClient(
+        metadataMetaClient.getFs(), metadataMetaClient.getBasePathV2().toString(),
+        CLIENT_HEARTBEAT_INTERVAL_IN_MS.defaultValue().longValue(),
+        CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES.defaultValue());
+    heartbeatClient.start(mdtCommitTime);
+
+    upsertToTable(metadataConfig, tableName);
+    metaClient = reload(metaClient);
+    metadataMetaClient = reload(metadataMetaClient);
+    // The delta commit from async indexer in metadata table should not be rolled back
+    assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.getTimestamp()));
+    assertTrue(metadataMetaClient.getActiveTimeline().getRollbackTimeline().empty());
+
+    // Simulate heartbeat timeout
+    heartbeatClient.stop(mdtCommitTime);
+    upsertToTable(metadataConfig, tableName);
+    metaClient = reload(metaClient);
+    metadataMetaClient = reload(metadataMetaClient);
+    // The delta commit from async indexer in metadata table should be rolled back now
+    assertFalse(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.getTimestamp()));
+    assertEquals(1, metadataMetaClient.getActiveTimeline().getRollbackTimeline().countInstants());
+    HoodieInstant rollbackInstant = metadataMetaClient.getActiveTimeline()
+        .getRollbackTimeline().firstInstant().get();
+    HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(
+        metadataMetaClient.getActiveTimeline().readRollbackInfoAsBytes(rollbackInstant).get());
+    assertEquals(mdtCommitTime, rollbackMetadata.getInstantsRollback()
+        .stream().findFirst().get().getCommitTime());
+  }
+
   private static Stream<Arguments> colStatsFileGroupCountParams() {
     return Stream.of(
         Arguments.of(1),
@@ -163,7 +245,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     String tableName = "indexer_test";
     // enable files and bloom_filters on the regular write client
     HoodieMetadataConfig.Builder metadataConfigBuilder = getMetadataConfigBuilder(false, false).withMetadataIndexBloomFilter(true);
-    initializeWriteClient(metadataConfigBuilder.build(), tableName);
+    upsertToTable(metadataConfigBuilder.build(), tableName);
 
     // validate table config
     assertFalse(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
@@ -190,7 +272,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     String tableName = "indexer_test";
     // enable files and bloom_filters on the regular write client
     HoodieMetadataConfig.Builder metadataConfigBuilder = getMetadataConfigBuilder(false, false);
-    initializeWriteClient(metadataConfigBuilder.build(), tableName);
+    upsertToTable(metadataConfigBuilder.build(), tableName);
     // validate table config
     assertFalse(reload(metaClient).getTableConfig().getMetadataPartitions().contains(FILES.getPartitionPath()));
 
@@ -227,12 +309,12 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     assertEquals(partitionFileSlices.size(), HoodieMetadataConfig.METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT.defaultValue());
   }
 
-  private void initializeWriteClient(HoodieMetadataConfig metadataConfig, String tableName) {
+  private void upsertToTable(HoodieMetadataConfig metadataConfig, String tableName) {
     HoodieWriteConfig.Builder writeConfigBuilder = getWriteConfigBuilder(basePath(), tableName);
     HoodieWriteConfig writeConfig = writeConfigBuilder.withMetadataConfig(metadataConfig).build();
     // do one upsert with synchronous metadata update
     SparkRDDWriteClient writeClient = new SparkRDDWriteClient(context(), writeConfig);
-    String instant = "0001";
+    String instant = HoodieActiveTimeline.createNewInstantTime();
     writeClient.startCommitWithTime(instant);
     List<HoodieRecord> records = DATA_GENERATOR.generateInserts(instant, 100);
     JavaRDD<WriteStatus> result = writeClient.upsert(jsc().parallelize(records, 1), instant);
@@ -240,8 +322,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     assertNoWriteErrors(statuses);
   }
 
-  private void indexMetadataPartitionsAndAssert(MetadataPartitionType partitionTypeToIndex, List<MetadataPartitionType> alreadyCompletedPartitions, List<MetadataPartitionType> nonExistantPartitions,
-                                                String tableName) {
+  private void scheduleAndExecuteIndexing(MetadataPartitionType partitionTypeToIndex, String tableName) {
     HoodieIndexer.Config config = new HoodieIndexer.Config();
     String propsPath = Objects.requireNonNull(getClass().getClassLoader().getResource("delta-streamer-config/indexer.properties")).getPath();
     config.basePath = basePath();
@@ -258,6 +339,13 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
 
     // validate table config
     metaClient = reload(metaClient);
+  }
+
+  private void indexMetadataPartitionsAndAssert(MetadataPartitionType partitionTypeToIndex, List<MetadataPartitionType> alreadyCompletedPartitions, List<MetadataPartitionType> nonExistantPartitions,
+                                                String tableName) {
+    scheduleAndExecuteIndexing(partitionTypeToIndex, tableName);
+
+    // validate table config
     Set<String> completedPartitions = metaClient.getTableConfig().getMetadataPartitions();
     assertTrue(completedPartitions.contains(partitionTypeToIndex.getPartitionPath()));
     alreadyCompletedPartitions.forEach(entry -> assertTrue(completedPartitions.contains(entry.getPartitionPath())));
@@ -277,7 +365,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     HoodieWriteConfig writeConfig = writeConfigBuilder.withMetadataConfig(metadataConfigBuilder.build()).build();
     // do one upsert with synchronous metadata update
     SparkRDDWriteClient writeClient = new SparkRDDWriteClient(context(), writeConfig);
-    String instant = "0001";
+    String instant = HoodieActiveTimeline.createNewInstantTime();
     writeClient.startCommitWithTime(instant);
     List<HoodieRecord> records = DATA_GENERATOR.generateInserts(instant, 100);
     JavaRDD<WriteStatus> result = writeClient.upsert(jsc().parallelize(records, 1), instant);
@@ -331,7 +419,7 @@ public class TestHoodieIndexer extends SparkClientFunctionalTestHarness implemen
     HoodieWriteConfig writeConfig = writeConfigBuilder.withMetadataConfig(metadataConfigBuilder.build()).build();
     // do one upsert with synchronous metadata update
     SparkRDDWriteClient writeClient = new SparkRDDWriteClient(context(), writeConfig);
-    String instant = "0001";
+    String instant = HoodieActiveTimeline.createNewInstantTime();
     writeClient.startCommitWithTime(instant);
     List<HoodieRecord> records = DATA_GENERATOR.generateInserts(instant, 100);
     JavaRDD<WriteStatus> result = writeClient.upsert(jsc().parallelize(records, 1), instant);


[hudi] 05/05: Bumping release candidate number 3

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit fe664886029657eb2c2c303be18aaf1c598a7181
Author: Y Ethan Guo <yi...@apache.org>
AuthorDate: Sun Feb 12 22:22:38 2023 -0800

    Bumping release candidate number 3
---
 docker/hoodie/hadoop/base/pom.xml                                    | 2 +-
 docker/hoodie/hadoop/base_java11/pom.xml                             | 2 +-
 docker/hoodie/hadoop/datanode/pom.xml                                | 2 +-
 docker/hoodie/hadoop/historyserver/pom.xml                           | 2 +-
 docker/hoodie/hadoop/hive_base/pom.xml                               | 2 +-
 docker/hoodie/hadoop/namenode/pom.xml                                | 2 +-
 docker/hoodie/hadoop/pom.xml                                         | 2 +-
 docker/hoodie/hadoop/prestobase/pom.xml                              | 2 +-
 docker/hoodie/hadoop/spark_base/pom.xml                              | 2 +-
 docker/hoodie/hadoop/sparkadhoc/pom.xml                              | 2 +-
 docker/hoodie/hadoop/sparkmaster/pom.xml                             | 2 +-
 docker/hoodie/hadoop/sparkworker/pom.xml                             | 2 +-
 docker/hoodie/hadoop/trinobase/pom.xml                               | 2 +-
 docker/hoodie/hadoop/trinocoordinator/pom.xml                        | 2 +-
 docker/hoodie/hadoop/trinoworker/pom.xml                             | 2 +-
 hudi-aws/pom.xml                                                     | 4 ++--
 hudi-cli/pom.xml                                                     | 2 +-
 hudi-client/hudi-client-common/pom.xml                               | 4 ++--
 hudi-client/hudi-flink-client/pom.xml                                | 4 ++--
 hudi-client/hudi-java-client/pom.xml                                 | 4 ++--
 hudi-client/hudi-spark-client/pom.xml                                | 4 ++--
 hudi-client/pom.xml                                                  | 2 +-
 hudi-common/pom.xml                                                  | 2 +-
 hudi-examples/hudi-examples-common/pom.xml                           | 2 +-
 hudi-examples/hudi-examples-flink/pom.xml                            | 2 +-
 hudi-examples/hudi-examples-java/pom.xml                             | 2 +-
 hudi-examples/hudi-examples-spark/pom.xml                            | 2 +-
 hudi-examples/pom.xml                                                | 2 +-
 hudi-flink-datasource/hudi-flink/pom.xml                             | 4 ++--
 hudi-flink-datasource/hudi-flink1.13.x/pom.xml                       | 4 ++--
 hudi-flink-datasource/hudi-flink1.14.x/pom.xml                       | 4 ++--
 hudi-flink-datasource/hudi-flink1.15.x/pom.xml                       | 4 ++--
 hudi-flink-datasource/hudi-flink1.16.x/pom.xml                       | 4 ++--
 hudi-flink-datasource/pom.xml                                        | 4 ++--
 hudi-gcp/pom.xml                                                     | 2 +-
 hudi-hadoop-mr/pom.xml                                               | 2 +-
 hudi-integ-test/pom.xml                                              | 2 +-
 hudi-kafka-connect/pom.xml                                           | 4 ++--
 hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml | 2 +-
 hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml | 2 +-
 hudi-platform-service/hudi-metaserver/pom.xml                        | 4 ++--
 hudi-platform-service/pom.xml                                        | 2 +-
 hudi-spark-datasource/hudi-spark-common/pom.xml                      | 4 ++--
 hudi-spark-datasource/hudi-spark/pom.xml                             | 4 ++--
 hudi-spark-datasource/hudi-spark2-common/pom.xml                     | 2 +-
 hudi-spark-datasource/hudi-spark2/pom.xml                            | 4 ++--
 hudi-spark-datasource/hudi-spark3-common/pom.xml                     | 2 +-
 hudi-spark-datasource/hudi-spark3.1.x/pom.xml                        | 4 ++--
 hudi-spark-datasource/hudi-spark3.2.x/pom.xml                        | 4 ++--
 hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml               | 2 +-
 hudi-spark-datasource/hudi-spark3.3.x/pom.xml                        | 4 ++--
 hudi-spark-datasource/pom.xml                                        | 2 +-
 hudi-sync/hudi-adb-sync/pom.xml                                      | 2 +-
 hudi-sync/hudi-datahub-sync/pom.xml                                  | 2 +-
 hudi-sync/hudi-hive-sync/pom.xml                                     | 2 +-
 hudi-sync/hudi-sync-common/pom.xml                                   | 2 +-
 hudi-sync/pom.xml                                                    | 2 +-
 hudi-tests-common/pom.xml                                            | 2 +-
 hudi-timeline-service/pom.xml                                        | 2 +-
 hudi-utilities/pom.xml                                               | 2 +-
 packaging/hudi-aws-bundle/pom.xml                                    | 2 +-
 packaging/hudi-cli-bundle/pom.xml                                    | 2 +-
 packaging/hudi-datahub-sync-bundle/pom.xml                           | 2 +-
 packaging/hudi-flink-bundle/pom.xml                                  | 2 +-
 packaging/hudi-gcp-bundle/pom.xml                                    | 2 +-
 packaging/hudi-hadoop-mr-bundle/pom.xml                              | 2 +-
 packaging/hudi-hive-sync-bundle/pom.xml                              | 2 +-
 packaging/hudi-integ-test-bundle/pom.xml                             | 2 +-
 packaging/hudi-kafka-connect-bundle/pom.xml                          | 2 +-
 packaging/hudi-metaserver-server-bundle/pom.xml                      | 2 +-
 packaging/hudi-presto-bundle/pom.xml                                 | 2 +-
 packaging/hudi-spark-bundle/pom.xml                                  | 2 +-
 packaging/hudi-timeline-server-bundle/pom.xml                        | 2 +-
 packaging/hudi-trino-bundle/pom.xml                                  | 2 +-
 packaging/hudi-utilities-bundle/pom.xml                              | 2 +-
 packaging/hudi-utilities-slim-bundle/pom.xml                         | 2 +-
 pom.xml                                                              | 2 +-
 77 files changed, 96 insertions(+), 96 deletions(-)

diff --git a/docker/hoodie/hadoop/base/pom.xml b/docker/hoodie/hadoop/base/pom.xml
index b8744c24e55..7df57e513c8 100644
--- a/docker/hoodie/hadoop/base/pom.xml
+++ b/docker/hoodie/hadoop/base/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/base_java11/pom.xml b/docker/hoodie/hadoop/base_java11/pom.xml
index 89873bf7110..2208d1424e8 100644
--- a/docker/hoodie/hadoop/base_java11/pom.xml
+++ b/docker/hoodie/hadoop/base_java11/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/datanode/pom.xml b/docker/hoodie/hadoop/datanode/pom.xml
index f215f67c076..302dddaa0b4 100644
--- a/docker/hoodie/hadoop/datanode/pom.xml
+++ b/docker/hoodie/hadoop/datanode/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/historyserver/pom.xml b/docker/hoodie/hadoop/historyserver/pom.xml
index c4eb4b0c1cb..108d9b37721 100644
--- a/docker/hoodie/hadoop/historyserver/pom.xml
+++ b/docker/hoodie/hadoop/historyserver/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml
index f9a15fc2621..29fe350c616 100644
--- a/docker/hoodie/hadoop/hive_base/pom.xml
+++ b/docker/hoodie/hadoop/hive_base/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/namenode/pom.xml b/docker/hoodie/hadoop/namenode/pom.xml
index 5e46fce0e99..7162fbd2a3d 100644
--- a/docker/hoodie/hadoop/namenode/pom.xml
+++ b/docker/hoodie/hadoop/namenode/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/pom.xml b/docker/hoodie/hadoop/pom.xml
index f9f4322747e..92fab9907d6 100644
--- a/docker/hoodie/hadoop/pom.xml
+++ b/docker/hoodie/hadoop/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/docker/hoodie/hadoop/prestobase/pom.xml b/docker/hoodie/hadoop/prestobase/pom.xml
index 5cba06dfffa..0f6c3dbeff7 100644
--- a/docker/hoodie/hadoop/prestobase/pom.xml
+++ b/docker/hoodie/hadoop/prestobase/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/spark_base/pom.xml b/docker/hoodie/hadoop/spark_base/pom.xml
index bd8b7bb3909..eb311d2ad11 100644
--- a/docker/hoodie/hadoop/spark_base/pom.xml
+++ b/docker/hoodie/hadoop/spark_base/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/sparkadhoc/pom.xml b/docker/hoodie/hadoop/sparkadhoc/pom.xml
index 71e36799f0b..ab8d3f6fafc 100644
--- a/docker/hoodie/hadoop/sparkadhoc/pom.xml
+++ b/docker/hoodie/hadoop/sparkadhoc/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/sparkmaster/pom.xml b/docker/hoodie/hadoop/sparkmaster/pom.xml
index cee6d960819..4c6e189f7bc 100644
--- a/docker/hoodie/hadoop/sparkmaster/pom.xml
+++ b/docker/hoodie/hadoop/sparkmaster/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/sparkworker/pom.xml b/docker/hoodie/hadoop/sparkworker/pom.xml
index a2e6a4edad7..6583911170a 100644
--- a/docker/hoodie/hadoop/sparkworker/pom.xml
+++ b/docker/hoodie/hadoop/sparkworker/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi-hadoop-docker</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/trinobase/pom.xml b/docker/hoodie/hadoop/trinobase/pom.xml
index afc51b840f2..21de2e4832b 100644
--- a/docker/hoodie/hadoop/trinobase/pom.xml
+++ b/docker/hoodie/hadoop/trinobase/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>hudi-hadoop-docker</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/trinocoordinator/pom.xml b/docker/hoodie/hadoop/trinocoordinator/pom.xml
index f0452e9c1bb..4a5938ad025 100644
--- a/docker/hoodie/hadoop/trinocoordinator/pom.xml
+++ b/docker/hoodie/hadoop/trinocoordinator/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>hudi-hadoop-docker</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <packaging>pom</packaging>
diff --git a/docker/hoodie/hadoop/trinoworker/pom.xml b/docker/hoodie/hadoop/trinoworker/pom.xml
index ddcc7b40a61..f240472d5e8 100644
--- a/docker/hoodie/hadoop/trinoworker/pom.xml
+++ b/docker/hoodie/hadoop/trinoworker/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <artifactId>hudi-hadoop-docker</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <packaging>pom</packaging>
diff --git a/hudi-aws/pom.xml b/hudi-aws/pom.xml
index bc07be24e4c..bd4f61a62a2 100644
--- a/hudi-aws/pom.xml
+++ b/hudi-aws/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-aws</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
 
     <name>hudi-aws</name>
     <packaging>jar</packaging>
diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index 32d4d38120b..ff178502326 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml
index 9b417b9c3cc..f25cc80ffce 100644
--- a/hudi-client/hudi-client-common/pom.xml
+++ b/hudi-client/hudi-client-common/pom.xml
@@ -19,12 +19,12 @@
   <parent>
     <artifactId>hudi-client</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-client-common</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-client-common</name>
   <packaging>jar</packaging>
diff --git a/hudi-client/hudi-flink-client/pom.xml b/hudi-client/hudi-flink-client/pom.xml
index a4b7f4be717..0618dc0fea2 100644
--- a/hudi-client/hudi-flink-client/pom.xml
+++ b/hudi-client/hudi-flink-client/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-client</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink-client</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
 
     <name>hudi-flink-client</name>
     <packaging>jar</packaging>
diff --git a/hudi-client/hudi-java-client/pom.xml b/hudi-client/hudi-java-client/pom.xml
index 35127a1be83..065c6d76027 100644
--- a/hudi-client/hudi-java-client/pom.xml
+++ b/hudi-client/hudi-java-client/pom.xml
@@ -19,12 +19,12 @@
     <parent>
         <artifactId>hudi-client</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-java-client</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
 
     <name>hudi-java-client</name>
     <packaging>jar</packaging>
diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml
index 9fdbb7054d2..aedf587cb0e 100644
--- a/hudi-client/hudi-spark-client/pom.xml
+++ b/hudi-client/hudi-spark-client/pom.xml
@@ -19,12 +19,12 @@
   <parent>
     <artifactId>hudi-client</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark-client</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark-client</name>
   <packaging>jar</packaging>
diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml
index b44d602a863..704215d01c3 100644
--- a/hudi-client/pom.xml
+++ b/hudi-client/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml
index 6999612f51e..7fae8edd447 100644
--- a/hudi-common/pom.xml
+++ b/hudi-common/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-examples/hudi-examples-common/pom.xml b/hudi-examples/hudi-examples-common/pom.xml
index 3f7f84d70b3..9eb3a37504b 100644
--- a/hudi-examples/hudi-examples-common/pom.xml
+++ b/hudi-examples/hudi-examples-common/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-examples</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-examples/hudi-examples-flink/pom.xml b/hudi-examples/hudi-examples-flink/pom.xml
index ee326b83eb9..c160e0a77ca 100644
--- a/hudi-examples/hudi-examples-flink/pom.xml
+++ b/hudi-examples/hudi-examples-flink/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-examples</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-examples/hudi-examples-java/pom.xml b/hudi-examples/hudi-examples-java/pom.xml
index 27256aa6bfe..68055949ea3 100644
--- a/hudi-examples/hudi-examples-java/pom.xml
+++ b/hudi-examples/hudi-examples-java/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-examples</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-examples/hudi-examples-spark/pom.xml b/hudi-examples/hudi-examples-spark/pom.xml
index 737b777e754..8d96f529d0d 100644
--- a/hudi-examples/hudi-examples-spark/pom.xml
+++ b/hudi-examples/hudi-examples-spark/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-examples</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml
index 0a1a66abd62..aa599ed431a 100644
--- a/hudi-examples/pom.xml
+++ b/hudi-examples/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-flink-datasource/hudi-flink/pom.xml b/hudi-flink-datasource/hudi-flink/pom.xml
index ea25c5ef486..2644cbcfcbd 100644
--- a/hudi-flink-datasource/hudi-flink/pom.xml
+++ b/hudi-flink-datasource/hudi-flink/pom.xml
@@ -22,12 +22,12 @@
     <parent>
         <artifactId>hudi-flink-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml
index 715636218c8..aa6d318a3f0 100644
--- a/hudi-flink-datasource/hudi-flink1.13.x/pom.xml
+++ b/hudi-flink-datasource/hudi-flink1.13.x/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-flink-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink1.13.x</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml
index 8028388d6ce..70083ece1f6 100644
--- a/hudi-flink-datasource/hudi-flink1.14.x/pom.xml
+++ b/hudi-flink-datasource/hudi-flink1.14.x/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-flink-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink1.14.x</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml
index 4a1f22d0de6..88e159ac3ff 100644
--- a/hudi-flink-datasource/hudi-flink1.15.x/pom.xml
+++ b/hudi-flink-datasource/hudi-flink1.15.x/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-flink-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink1.15.x</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml
index bed6fb08538..ce85ad45f8e 100644
--- a/hudi-flink-datasource/hudi-flink1.16.x/pom.xml
+++ b/hudi-flink-datasource/hudi-flink1.16.x/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-flink-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink1.16.x</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-flink-datasource/pom.xml b/hudi-flink-datasource/pom.xml
index a81182fb71a..9294b3eabfc 100644
--- a/hudi-flink-datasource/pom.xml
+++ b/hudi-flink-datasource/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-flink-datasource</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>pom</packaging>
 
     <properties>
diff --git a/hudi-gcp/pom.xml b/hudi-gcp/pom.xml
index 9fe84ea85ab..7ac3db8e2be 100644
--- a/hudi-gcp/pom.xml
+++ b/hudi-gcp/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml
index 144b54c2233..d94b75f5dbf 100644
--- a/hudi-hadoop-mr/pom.xml
+++ b/hudi-hadoop-mr/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml
index 93692f9f3a8..e0cc71f41cc 100644
--- a/hudi-integ-test/pom.xml
+++ b/hudi-integ-test/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
   <artifactId>hudi-integ-test</artifactId>
diff --git a/hudi-kafka-connect/pom.xml b/hudi-kafka-connect/pom.xml
index 071ea07552a..c3fc94f2458 100644
--- a/hudi-kafka-connect/pom.xml
+++ b/hudi-kafka-connect/pom.xml
@@ -19,13 +19,13 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-kafka-connect</artifactId>
     <description>Kafka Connect Sink Connector for Hudi</description>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <packaging>jar</packaging>
 
     <properties>
diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml
index 8dc8f1ffc02..2bf6cc388c4 100644
--- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml
+++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-metaserver</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml
index fbc788eab11..045b41752aa 100644
--- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml
+++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-metaserver</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-platform-service/hudi-metaserver/pom.xml b/hudi-platform-service/hudi-metaserver/pom.xml
index 68201d95e64..c5e9975264b 100644
--- a/hudi-platform-service/hudi-metaserver/pom.xml
+++ b/hudi-platform-service/hudi-metaserver/pom.xml
@@ -20,12 +20,12 @@
     <parent>
         <artifactId>hudi-platform-service</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
     <artifactId>hudi-metaserver</artifactId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
 
     <name>hudi-metaserver</name>
     <packaging>pom</packaging>
diff --git a/hudi-platform-service/pom.xml b/hudi-platform-service/pom.xml
index b24ddadac92..fb95b154122 100644
--- a/hudi-platform-service/pom.xml
+++ b/hudi-platform-service/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-spark-datasource/hudi-spark-common/pom.xml b/hudi-spark-datasource/hudi-spark-common/pom.xml
index 2ca905bc026..4fd34a9a546 100644
--- a/hudi-spark-datasource/hudi-spark-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark-common/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark-common_${scala.binary.version}</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/hudi-spark/pom.xml b/hudi-spark-datasource/hudi-spark/pom.xml
index 8331c6c84ed..f7ecf351813 100644
--- a/hudi-spark-datasource/hudi-spark/pom.xml
+++ b/hudi-spark-datasource/hudi-spark/pom.xml
@@ -19,12 +19,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark_${scala.binary.version}</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark_${scala.binary.version}</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/hudi-spark2-common/pom.xml b/hudi-spark-datasource/hudi-spark2-common/pom.xml
index 584613eac9f..61241a12a9b 100644
--- a/hudi-spark-datasource/hudi-spark2-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark2-common/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-spark-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-spark-datasource/hudi-spark2/pom.xml b/hudi-spark-datasource/hudi-spark2/pom.xml
index 892d8110931..59f81dac456 100644
--- a/hudi-spark-datasource/hudi-spark2/pom.xml
+++ b/hudi-spark-datasource/hudi-spark2/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark2_${scala.binary.version}</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark2_${scala.binary.version}</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/hudi-spark3-common/pom.xml b/hudi-spark-datasource/hudi-spark3-common/pom.xml
index 7001191aaa3..e1be6740151 100644
--- a/hudi-spark-datasource/hudi-spark3-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3-common/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-spark-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
index d259ed5dc24..dd7507f570a 100644
--- a/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.1.x/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark3.1.x_2.12</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark3.1.x_2.12</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
index 1af844c4b60..3771e75f052 100644
--- a/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.2.x/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark3.2.x_2.12</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark3.2.x_2.12</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml
index b137e8cfaea..b4366feee7b 100644
--- a/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.2plus-common/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi-spark-datasource</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
index a112842dd8b..ff25b5e0ae1 100644
--- a/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
+++ b/hudi-spark-datasource/hudi-spark3.3.x/pom.xml
@@ -17,12 +17,12 @@
   <parent>
     <artifactId>hudi-spark-datasource</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-spark3.3.x_2.12</artifactId>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
 
   <name>hudi-spark3.3.x_2.12</name>
   <packaging>jar</packaging>
diff --git a/hudi-spark-datasource/pom.xml b/hudi-spark-datasource/pom.xml
index 102a654e756..e437820b5eb 100644
--- a/hudi-spark-datasource/pom.xml
+++ b/hudi-spark-datasource/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-sync/hudi-adb-sync/pom.xml b/hudi-sync/hudi-adb-sync/pom.xml
index b5306f38162..fe58a1d1866 100644
--- a/hudi-sync/hudi-adb-sync/pom.xml
+++ b/hudi-sync/hudi-adb-sync/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/hudi-sync/hudi-datahub-sync/pom.xml b/hudi-sync/hudi-datahub-sync/pom.xml
index b46b6e28cf7..9dbf6d2f176 100644
--- a/hudi-sync/hudi-datahub-sync/pom.xml
+++ b/hudi-sync/hudi-datahub-sync/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/hudi-sync/hudi-hive-sync/pom.xml b/hudi-sync/hudi-hive-sync/pom.xml
index 8992636b613..fb9645578ca 100644
--- a/hudi-sync/hudi-hive-sync/pom.xml
+++ b/hudi-sync/hudi-hive-sync/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 
diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml
index 79a99e28c53..d883d14e9a6 100644
--- a/hudi-sync/hudi-sync-common/pom.xml
+++ b/hudi-sync/hudi-sync-common/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/hudi-sync/pom.xml b/hudi-sync/pom.xml
index 745e1c1aaad..23991c9a062 100644
--- a/hudi-sync/pom.xml
+++ b/hudi-sync/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-tests-common/pom.xml b/hudi-tests-common/pom.xml
index 92e9aa4918d..ec397f34cff 100644
--- a/hudi-tests-common/pom.xml
+++ b/hudi-tests-common/pom.xml
@@ -17,7 +17,7 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
     </parent>
 
     <modelVersion>4.0.0</modelVersion>
diff --git a/hudi-timeline-service/pom.xml b/hudi-timeline-service/pom.xml
index 3377dd728c7..0968d71678e 100644
--- a/hudi-timeline-service/pom.xml
+++ b/hudi-timeline-service/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index 81fb899fc39..2a91458db8b 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
 
diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml
index 5e3d20eab30..0e2355da506 100644
--- a/packaging/hudi-aws-bundle/pom.xml
+++ b/packaging/hudi-aws-bundle/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-cli-bundle/pom.xml b/packaging/hudi-cli-bundle/pom.xml
index cb1c174c32b..10e35c7dbe8 100644
--- a/packaging/hudi-cli-bundle/pom.xml
+++ b/packaging/hudi-cli-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-datahub-sync-bundle/pom.xml b/packaging/hudi-datahub-sync-bundle/pom.xml
index 46ddd63922e..ef6b029ab8c 100644
--- a/packaging/hudi-datahub-sync-bundle/pom.xml
+++ b/packaging/hudi-datahub-sync-bundle/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml
index 4ec8d7df3b4..7ea45bb78f9 100644
--- a/packaging/hudi-flink-bundle/pom.xml
+++ b/packaging/hudi-flink-bundle/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-gcp-bundle/pom.xml b/packaging/hudi-gcp-bundle/pom.xml
index 7a47841de98..c2fdda0cd99 100644
--- a/packaging/hudi-gcp-bundle/pom.xml
+++ b/packaging/hudi-gcp-bundle/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml
index feaebc39d9a..7adfbc605e2 100644
--- a/packaging/hudi-hadoop-mr-bundle/pom.xml
+++ b/packaging/hudi-hadoop-mr-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-hive-sync-bundle/pom.xml b/packaging/hudi-hive-sync-bundle/pom.xml
index 5b79eed28f9..f548410dcc1 100644
--- a/packaging/hudi-hive-sync-bundle/pom.xml
+++ b/packaging/hudi-hive-sync-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml
index 4b4c512978c..f9aa5e76f2f 100644
--- a/packaging/hudi-integ-test-bundle/pom.xml
+++ b/packaging/hudi-integ-test-bundle/pom.xml
@@ -17,7 +17,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-kafka-connect-bundle/pom.xml b/packaging/hudi-kafka-connect-bundle/pom.xml
index aa6b7cd0037..d3616a07107 100644
--- a/packaging/hudi-kafka-connect-bundle/pom.xml
+++ b/packaging/hudi-kafka-connect-bundle/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-metaserver-server-bundle/pom.xml b/packaging/hudi-metaserver-server-bundle/pom.xml
index 9f077c3e2b2..5344e7b9e8a 100644
--- a/packaging/hudi-metaserver-server-bundle/pom.xml
+++ b/packaging/hudi-metaserver-server-bundle/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml
index 23a3ebca506..12865af3ec4 100644
--- a/packaging/hudi-presto-bundle/pom.xml
+++ b/packaging/hudi-presto-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index 8724fe42b5c..66741c1a57a 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-timeline-server-bundle/pom.xml b/packaging/hudi-timeline-server-bundle/pom.xml
index 6d6130d57c9..7d2acc78a45 100644
--- a/packaging/hudi-timeline-server-bundle/pom.xml
+++ b/packaging/hudi-timeline-server-bundle/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>hudi</artifactId>
         <groupId>org.apache.hudi</groupId>
-        <version>0.13.0-rc2</version>
+        <version>0.13.0-rc3</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-trino-bundle/pom.xml b/packaging/hudi-trino-bundle/pom.xml
index e607f3992b3..9435f5f944e 100644
--- a/packaging/hudi-trino-bundle/pom.xml
+++ b/packaging/hudi-trino-bundle/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml
index 3085a3b4eb9..4fc4974096a 100644
--- a/packaging/hudi-utilities-bundle/pom.xml
+++ b/packaging/hudi-utilities-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/packaging/hudi-utilities-slim-bundle/pom.xml b/packaging/hudi-utilities-slim-bundle/pom.xml
index dbbb1646694..c98dafcac73 100644
--- a/packaging/hudi-utilities-slim-bundle/pom.xml
+++ b/packaging/hudi-utilities-slim-bundle/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hudi</artifactId>
     <groupId>org.apache.hudi</groupId>
-    <version>0.13.0-rc2</version>
+    <version>0.13.0-rc3</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
diff --git a/pom.xml b/pom.xml
index 259c8de404d..405e24c5e53 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,7 +29,7 @@
   <groupId>org.apache.hudi</groupId>
   <artifactId>hudi</artifactId>
   <packaging>pom</packaging>
-  <version>0.13.0-rc2</version>
+  <version>0.13.0-rc3</version>
   <description>Apache Hudi brings stream style processing on big data</description>
   <url>https://github.com/apache/hudi</url>
   <name>Hudi</name>


[hudi] 01/05: [HUDI-5758] Restoring state of `HoodieKey` to make sure it's binary compatible w/ its state in 0.12 (#7917)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 847e7a975bfeb94956885cc252285f95afc4a843
Author: Alexey Kudinkin <al...@gmail.com>
AuthorDate: Fri Feb 10 15:02:47 2023 -0800

    [HUDI-5758] Restoring state of `HoodieKey` to make sure it's binary compatible w/ its state in 0.12 (#7917)
    
    RFC-46 modified `HoodieKey` to substantially optimize its serialized footprint (while using Kryo) by making it explicitly serializable by Kryo (inheriting form `KryoSerializable`, making it final).
    
    However, this broken its binary compatibility w/ the state as it was in 0.12.2.
    
    Unfortunately, this entailed that as this class is used in `DeleteRecord` w/in `HoodieDeleteBlock` that it also made impossible to read such blocks created by prior Hudi versions (more details in HUDI-5758).
    
    This PR restores previous state for `HoodieKey` to make sure it stays binary compatible w/ existing persisted `HoodieDeleteBlock` created by prior Hudi versions
---
 .../apache/spark/HoodieSparkKryoRegistrar.scala    | 25 +++++++++++++++++--
 .../org/apache/hudi/common/model/DeleteRecord.java |  9 +++++++
 .../org/apache/hudi/common/model/HoodieKey.java    | 28 ++++++++--------------
 .../common/table/log/block/HoodieDeleteBlock.java  |  2 ++
 4 files changed, 44 insertions(+), 20 deletions(-)

diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
index 3894065d809..9d7fa3b784f 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/HoodieSparkKryoRegistrar.scala
@@ -18,11 +18,12 @@
 
 package org.apache.spark
 
-import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.esotericsoftware.kryo.{Kryo, Serializer}
 import com.esotericsoftware.kryo.serializers.JavaSerializer
 import org.apache.hudi.client.model.HoodieInternalRow
 import org.apache.hudi.common.config.SerializableConfiguration
-import org.apache.hudi.common.model.HoodieSparkRecord
+import org.apache.hudi.common.model.{HoodieKey, HoodieSparkRecord}
 import org.apache.hudi.common.util.HoodieCommonKryoRegistrar
 import org.apache.hudi.config.HoodieWriteConfig
 import org.apache.spark.serializer.KryoRegistrator
@@ -44,12 +45,15 @@ import org.apache.spark.serializer.KryoRegistrator
  * </ol>
  */
 class HoodieSparkKryoRegistrar extends HoodieCommonKryoRegistrar with KryoRegistrator {
+
   override def registerClasses(kryo: Kryo): Unit = {
     ///////////////////////////////////////////////////////////////////////////
     // NOTE: DO NOT REORDER REGISTRATIONS
     ///////////////////////////////////////////////////////////////////////////
     super[HoodieCommonKryoRegistrar].registerClasses(kryo)
 
+    kryo.register(classOf[HoodieKey], new HoodieKeySerializer)
+
     kryo.register(classOf[HoodieWriteConfig])
 
     kryo.register(classOf[HoodieSparkRecord])
@@ -59,6 +63,23 @@ class HoodieSparkKryoRegistrar extends HoodieCommonKryoRegistrar with KryoRegist
     //       we're relying on [[SerializableConfiguration]] wrapper to work it around
     kryo.register(classOf[SerializableConfiguration], new JavaSerializer())
   }
+
+  /**
+   * NOTE: This {@link Serializer} could deserialize instance of {@link HoodieKey} serialized
+   *       by implicitly generated Kryo serializer (based on {@link com.esotericsoftware.kryo.serializers.FieldSerializer}
+   */
+  class HoodieKeySerializer extends Serializer[HoodieKey] {
+    override def write(kryo: Kryo, output: Output, key: HoodieKey): Unit = {
+      output.writeString(key.getRecordKey)
+      output.writeString(key.getPartitionPath)
+    }
+
+    override def read(kryo: Kryo, input: Input, klass: Class[HoodieKey]): HoodieKey = {
+      val recordKey = input.readString()
+      val partitionPath = input.readString()
+      new HoodieKey(recordKey, partitionPath)
+    }
+  }
 }
 
 object HoodieSparkKryoRegistrar {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java
index 003b591c20c..296e95e8bfa 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DeleteRecord.java
@@ -28,6 +28,15 @@ import java.util.Objects;
  * we need to keep the ordering val to combine with the data records when merging, or the data loss
  * may occur if there are intermediate deletions for the inputs
  * (a new INSERT comes after a DELETE in one input batch).
+ *
+ * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING
+ *
+ *       This class is serialized (using Kryo) as part of {@code HoodieDeleteBlock} to make
+ *       sure this stays backwards-compatible we can't MAKE ANY CHANGES TO THIS CLASS (add,
+ *       delete, reorder or change types of the fields in this class, make class final, etc)
+ *       as this would break its compatibility with already persisted blocks.
+ *
+ *       Check out HUDI-5760 for more details
  */
 public class DeleteRecord implements Serializable {
   private static final long serialVersionUID = 1L;
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java
index 5208cece1cb..436758f96f4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieKey.java
@@ -18,11 +18,6 @@
 
 package org.apache.hudi.common.model;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.KryoSerializable;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-
 import java.io.Serializable;
 import java.util.Objects;
 
@@ -31,8 +26,17 @@ import java.util.Objects;
  * <p>
  * - recordKey : a recordKey that acts as primary key for a record.
  * - partitionPath : the partition path of a record.
+ *
+ * NOTE: PLEASE READ CAREFULLY BEFORE CHANGING
+ *
+ *       This class is serialized (using Kryo) as part of {@code HoodieDeleteBlock} to make
+ *       sure this stays backwards-compatible we can't MAKE ANY CHANGES TO THIS CLASS (add,
+ *       delete, reorder or change types of the fields in this class, make class final, etc)
+ *       as this would break its compatibility with already persisted blocks.
+ *
+ *       Check out HUDI-5760 for more details
  */
-public final class HoodieKey implements Serializable, KryoSerializable {
+public class HoodieKey implements Serializable {
 
   private String recordKey;
   private String partitionPath;
@@ -86,16 +90,4 @@ public final class HoodieKey implements Serializable, KryoSerializable {
     sb.append('}');
     return sb.toString();
   }
-
-  @Override
-  public void write(Kryo kryo, Output output) {
-    output.writeString(recordKey);
-    output.writeString(partitionPath);
-  }
-
-  @Override
-  public void read(Kryo kryo, Input input) {
-    this.recordKey = input.readString();
-    this.partitionPath = input.readString();
-  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java
index a5168072d01..d920495f971 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDeleteBlock.java
@@ -68,6 +68,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
 
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream output = new DataOutputStream(baos);
+    // TODO(HUDI-5760) avoid using Kryo for serialization here
     byte[] bytesToWrite = SerializationUtils.serialize(getRecordsToDelete());
     output.writeInt(version);
     output.writeInt(bytesToWrite.length);
@@ -97,6 +98,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
     }
   }
 
+  // TODO(HUDI-5760) avoid using Kryo for serialization here
   private static DeleteRecord[] deserialize(int version, byte[] data) {
     if (version == 1) {
       // legacy version


[hudi] 02/05: [HUDI-5768] Fix Spark Datasource read of metadata table (#7924)

Posted by yi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.13.0
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 7ccf6e678278ceca592b8d95160bb0b17906928f
Author: Y Ethan Guo <et...@gmail.com>
AuthorDate: Sun Feb 12 03:25:51 2023 -0800

    [HUDI-5768] Fix Spark Datasource read of metadata table (#7924)
---
 .../src/main/scala/org/apache/hudi/HoodieBaseRelation.scala          | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

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 bf3d38b808d..8a730a8334b 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
@@ -42,6 +42,7 @@ import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
 import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
 import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
 import org.apache.hudi.io.storage.HoodieAvroHFileReader
+import org.apache.hudi.metadata.HoodieTableMetadata
 import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
 import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.RDD
@@ -59,7 +60,6 @@ import org.apache.spark.sql.{Row, SQLContext, SparkSession}
 import org.apache.spark.unsafe.types.UTF8String
 
 import java.net.URI
-import java.util.Locale
 import scala.collection.JavaConverters._
 import scala.util.control.NonFatal
 import scala.util.{Failure, Success, Try}
@@ -292,7 +292,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
    * Determines whether relation's schema could be pruned by Spark's Optimizer
    */
   def canPruneRelationSchema: Boolean =
-    (fileFormat.isInstanceOf[ParquetFileFormat] || fileFormat.isInstanceOf[OrcFileFormat]) &&
+    !HoodieTableMetadata.isMetadataTable(basePath.toString) &&
+      (fileFormat.isInstanceOf[ParquetFileFormat] || fileFormat.isInstanceOf[OrcFileFormat]) &&
       // NOTE: In case this relation has already been pruned there's no point in pruning it again
       prunedDataSchema.isEmpty &&
       // TODO(HUDI-5421) internal schema doesn't support nested schema pruning currently