You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/09/30 03:35:17 UTC

[GitHub] [hudi] nsivabalan opened a new pull request, #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

nsivabalan opened a new pull request, #6836:
URL: https://github.com/apache/hudi/pull/6836

   ### Change Logs
   
   When metadata table is just getting initialized, but first commit is not yet fully complete, reading from metadata table could fail w/ below stacktrace. 
   
   Call trace that could result in this. 
   ```
   BaseHoodieTableFileIndex.doRefresh() // metadata Config will have metadata enabled if user enables for the query session. lets assume user enabled while the metadata table is being built out. 
   {
           HoodieTableMetadata newTableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, ....);
                  HoodieTableMetadata.create eventually will call constructor of HoodieBackedTableMetadata()
                           within which we call initIfNeeded()
                                           within initIfNeeded { 
                                                  we disable metadata only if table itself is not found. if not, metadata is still enabled. 
                                            }
   
         .
          -> loadPartitionPathFiles
   }
   
   loadPartitionPathFiles {
       ...
       getAllFilesInPartitionsUnchecked()
   }
   
   getAllFilesInPartitionsUnchecked {
        tableMetadata.getAllFilesInPartitions(list of interested partitions)
   }
   
   getAllFilesInPartitions{
       BaseTableMetadata.fetchAllFilesInPartitionPaths...
   }
   
   BaseTableMetadata.fetchAllFilesInPartitionPaths{
      ..
      getRecordsByKeys(...)
   }
   
   HoodieBackedTableMetadata.getRecordsByKeys{
       getPartitionFileSliceToKeysMapping()
   }
   
   getPartitionFileSliceToKeysMapping{
          List<FileSlice> latestFileSlices =
           HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
   }
   
   HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices {
      HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient));
       Stream<FileSlice> fileSliceStream;
       if (mergeFileSlices) { // this is true for this call graph. 
         if (metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) {
           fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(
               partition, metaClient.getActiveTimeline().filterCompletedInstants().**lastInstant().get()**.getTimestamp());
         }
   }
   ```
   
   There is no lastInstant as the Metadata table is still being initialized. 
   
   
   
   ### Impact
   
   _Describe any public API or user-facing feature change or any performance impact._
   
   **Risk level: none | low | medium | high**
   
   _Choose one. If medium or high, explain what verification was done to mitigate the risks._
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
     ticket number here and follow the [instruction](https://hudi.apache.org/contribute/developer-setup#website) to make
     changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6836:
URL: https://github.com/apache/hudi/pull/6836#discussion_r984839942


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1052,8 +1052,12 @@ private static List<FileSlice> getPartitionFileSlices(HoodieTableMetaClient meta
     HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient));
     Stream<FileSlice> fileSliceStream;
     if (mergeFileSlices) {
-      fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(
-          partition, metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp());
+      if (metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) {

Review Comment:
   We can do this more idiomatic:
   
   ```
   lastInstant.map(() -> {
     getLatestMergedSlices(partition, instant)
   }).getOrElse(emptyList())
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on a diff in pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on code in PR #6836:
URL: https://github.com/apache/hudi/pull/6836#discussion_r988495539


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -293,16 +293,20 @@ object HoodieFileIndex extends Logging {
     schema.fieldNames.filter { colName => refs.exists(r => resolver.apply(colName, r.name)) }
   }
 
-  def getConfigProperties(spark: SparkSession, options: Map[String, String]) = {
+  private def isFilesPartitionAvailable(metaClient: HoodieTableMetaClient): Boolean = {
+    metaClient.getTableConfig.getMetadataPartitions.contains(HoodieTableMetadataUtil.PARTITION_NAME_FILES)
+  }
+
+  def getConfigProperties(spark: SparkSession, options: Map[String, String], metaClient: HoodieTableMetaClient) = {
     val sqlConf: SQLConf = spark.sessionState.conf
     val properties = new TypedProperties()
 
     // To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
     // would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
-    properties.setProperty(HoodieMetadataConfig.ENABLE.key(),
-      sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
-        HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
-    properties.putAll(options.filter(p => p._2 != null).asJava)
+    val isMetadataFilesPartitionAvailable = isFilesPartitionAvailable(metaClient) && sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),

Review Comment:
   Isn't this the entry point to metadata table on the read path? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #6836:
URL: https://github.com/apache/hudi/pull/6836#discussion_r991629981


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala:
##########
@@ -403,7 +404,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
       val allFilesPartitions = fileIndex.listFiles(Seq(), Seq())
       assertEquals(10, allFilesPartitions.head.files.length)
 
-      if (testCase.enableDataSkipping && testCase.enableMetadata) {
+      if (testCase.enableDataSkipping && testCase.enableMetadata && testCase.enableColumnStats) {

Review Comment:
   Can you please elaborate why we need to change this one?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala:
##########
@@ -267,10 +266,15 @@ case class HoodieFileIndex(spark: SparkSession,
         s"(isMetadataTableEnabled = $isMetadataTableEnabled, isColumnStatsIndexEnabled = $isColumnStatsIndexEnabled")
     }
   }
+
 }
 
 object HoodieFileIndex extends Logging {
 
+  def getBooleanConfigValue(sqlConf: SQLConf, options: Map[String, String], configKey: String, defaultValue: String) : Boolean = {

Review Comment:
   Let's swap in SQLConf and opts order so that it matches the order of retrieving the values from them correspondingly



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1269231396

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 23d923e6b8c75781053f3f7bbc811084141f7786 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978) 
   * e246d65957362860b850f1af9ef973b85bf1a4eb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1272138058

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1273930030

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12115",
       "triggerID" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057) 
   * 8f763c8eacdf249b7ab74c98b0a163eedd7a9aec Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12115) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1263068964

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77223f8b87bdfcfa75045fb622b127cc4f9e47ab Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1269228467

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 23d923e6b8c75781053f3f7bbc811084141f7786 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978) 
   * e246d65957362860b850f1af9ef973b85bf1a4eb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1272058386

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e246d65957362860b850f1af9ef973b85bf1a4eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017) 
   * d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1263105795

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77223f8b87bdfcfa75045fb622b127cc4f9e47ab Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919) 
   * 34427d0e522bec7eee731644080bd0b5d20570dc Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1272095614

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e246d65957362860b850f1af9ef973b85bf1a4eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017) 
   * d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1269280015

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e246d65957362860b850f1af9ef973b85bf1a4eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1263280850

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 34427d0e522bec7eee731644080bd0b5d20570dc Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1263066749

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77223f8b87bdfcfa75045fb622b127cc4f9e47ab UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1273926369

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057) 
   * 8f763c8eacdf249b7ab74c98b0a163eedd7a9aec UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin merged pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
alexeykudinkin merged PR #6836:
URL: https://github.com/apache/hudi/pull/6836


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1273918590

   addressed all comments. will merge it in once CI is green


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1274009588

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11921",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11978",
       "triggerID" : "23d923e6b8c75781053f3f7bbc811084141f7786",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12017",
       "triggerID" : "e246d65957362860b850f1af9ef973b85bf1a4eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12057",
       "triggerID" : "d7fbaa4fed0c713ee0b0a8ba4b8900b11b89c433",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12115",
       "triggerID" : "8f763c8eacdf249b7ab74c98b0a163eedd7a9aec",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8f763c8eacdf249b7ab74c98b0a163eedd7a9aec Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12115) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #6836: [HUDI-4952] Fixing reading from metadata table when there are no inflight commits

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6836:
URL: https://github.com/apache/hudi/pull/6836#issuecomment-1263103453

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919",
       "triggerID" : "77223f8b87bdfcfa75045fb622b127cc4f9e47ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "34427d0e522bec7eee731644080bd0b5d20570dc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 77223f8b87bdfcfa75045fb622b127cc4f9e47ab Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11919) 
   * 34427d0e522bec7eee731644080bd0b5d20570dc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org