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/01/30 01:32:16 UTC

[GitHub] [hudi] XuQianJin-Stars opened a new pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

XuQianJin-Stars opened a new pull request #4720:
URL: https://github.com/apache/hudi/pull/4720


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *Support querying a table as of a savepoint*
   link: [HUDI-3221](https://issues.apache.org/jira/browse/HUDI-3221)
   Support Spark Version:
   
   | version | support |
   |  ------  | -------  |
   | 2.4.x     |    Yes     |
   | 3.1.2     |    Yes     |
   | 3.2.0    |    Yes     |
   | 3.0.x     |    No     |
   
   
   ## Brief change log
   
   *Delete partitions without metadata sync to hms*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026956456


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 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] xushiyan commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820171075



##########
File path: hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/adapter/Spark3Adapter.scala
##########
@@ -36,6 +36,9 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.hudi.SparkAdapter
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.hudi.SparkAdapter
+import org.apache.spark.sql.internal.SQLConf

Review comment:
       unused import?

##########
File path: hudi-spark-datasource/README.md
##########
@@ -36,3 +36,16 @@ file that supports spark sql on spark 2.x version.
 has no class since hudi only supports spark 2.4.4 version, and it acts as the placeholder when packaging hudi-spark-bundle module. 
 * hudi-spark3-common is the module that contains the code that would be reused between spark3.x versions.
 * hudi-spark-common is the module that contains the code that would be reused between spark2.x and spark3.x versions.
+
+## Description of Time Travel
+* `HoodieSpark3_2ExtendedSqlAstBuilder` have comments in the spark3.2's code fork from `org.apache.spark.sql.catalyst.parser.AstBuilder`, and additional `withTimeTravel` method.
+* `SqlBase.g4` have comments in the code forked from spark3.2's parser, and add SparkSQL Syntax  `TIMESTAMP AS OF` and `VERSION AS OF`.

Review comment:
       can you also list down which classes/files can be removed once upgrade to 3.3 ?

##########
File path: hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark3RowSerDe
+import org.apache.hudi.client.utils.SparkRowSerDe
+import org.apache.hudi.spark3.internal.ReflectUtil
+import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializer, HoodieAvroSerializerTrait, Spark3HoodieAvroDeserializer}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.JoinType
+import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, Join, JoinHint, LogicalPlan}
+import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, Spark3ParsePartitionUtil, SparkParsePartitionUtil}
+import org.apache.spark.sql.hudi.SparkAdapter
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.{Row, SparkSession}
+
+/**
+ * The adapter for spark3.

Review comment:
       ```suggestion
    * The adapter for spark3.2.
   ```

##########
File path: hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TimeTravelRelation.scala
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
+
+case class TimeTravelRelation(

Review comment:
       this looks identical to the one in `hudi-spark`. can we deduplicate?

##########
File path: hudi-spark-datasource/hudi-spark3/pom.xml
##########
@@ -157,7 +175,7 @@
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_2.12</artifactId>
-      <version>${spark3.version}</version>
+      <version>${spark3.2.version}</version>

Review comment:
       can you clarify why this change? what if user needs to build the project with spark 3.1.x profile?  

##########
File path: pom.xml
##########
@@ -120,6 +120,8 @@
     <flink.version>1.14.3</flink.version>
     <spark2.version>2.4.4</spark2.version>
     <spark3.version>3.2.1</spark3.version>
+    <spark3.1.version>3.1.2</spark3.1.version>
+    <spark3.2.version>3.2.1</spark3.2.version>

Review comment:
       not sure why we need these new properties. `spark3.version` is always the default and point to the latest supported spark 3. and we shall build the project with `spark3.1.x` if we want `spark3.version` point to 3.1. can you clarify




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a 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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060112762


   
   
   
   
   > Also please note that this error is not from a spark environment, but from AwsAthena, which uses prestodb as a engine.
   > 
   > There is any additional setup?
   
   prestodb also needs to support this syntax. 


-- 
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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060678330


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059943153


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059968749


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061629300


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) 
   
   <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] xushiyan edited a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan edited a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1062053812


   @fedsp it's a spark bundle or version mismatch problem where the syntax is not recognized. Maybe previous version from the branch has some misconfig but now it's resolved. I verified the feature in spark 3.2.1
   
   ```
   ➜ mvn -T 2.5C clean install -DskipTests -Djacoco.skip=true -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dscala-2.12 -Dspark3
   
   // test using packaging/hudi-spark-bundle/target/hudi-spark3.2.1-bundle_2.12-0.11.0-SNAPSHOT.jar
   
   // Spark 3.2.1
   // COW
   spark.sql("""
   create table hudi_cow_pt_tbl (
     id bigint,
     name string,
     ts bigint,
     dt string,
     hh string
   ) using hudi
   tblproperties (
     type = 'cow',
     primaryKey = 'id',
     preCombineField = 'ts'
    )
   partitioned by (dt, hh)
   location '/tmp/hudi/hudi_cow_pt_tbl';
   """)
   spark.sql("insert into hudi_cow_pt_tbl select 1, 'a0', 1000, '2021-12-09', '10'")
   spark.sql("select * from hudi_cow_pt_tbl").show()
   spark.sql("insert into hudi_cow_pt_tbl select 1, 'a1', 1001, '2021-12-09', '10'")
   spark.sql("select * from hudi_cow_pt_tbl").show()
   
   
   // time travel based on first commit time
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '20220308175415995' where id = 1").show()
   // time travel based on different timestamp formats
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '2022-03-08 17:54:15.995' where id = 1").show()
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '2022-03-09' where id = 1").show()
   ```
   
   


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040225571


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041336492


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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






-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048897563


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   * e0fadab437970eac2b15816d101db50586bc7d9b 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   * 9efcab8b8f103d07395badb779c875d76234717d 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041463330


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   
   <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] xushiyan commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048537726


   > @xushiyan , there is anything that I can help from my side to help you on this PR? I can help testing using real datasets
   
   hey @fedsp thank you for offering help. Feel free to build this branch against spark 3.2 (using maven profile spark3) and test with your datasets. Feel free to post any results or feedback. That'd be of great help! 


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048893828


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   * 53e78979a5a780834ce4a9bf34a9119544b0054f 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026837148


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   
   <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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060092313


   hi @fedsp Your spark version is 3.2.x? 


-- 
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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060099954


   Also please note that this error is not from a spark environment, but from AwsAthena, which uses prestodb as a engine.
   
   There is any additional setup?


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048963452


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060772423


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 82a888652bf47775fe2626c259129a367b38f766 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040165633


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   * 47745008de1658561623c4187330e1004794d253 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041034454


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   * 47745008de1658561623c4187330e1004794d253 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040195608






-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059908350


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   
   <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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060134821


   Thank you for the clarification @xushiyan! 
   If you are avaliable, I can test right now screensharing and we can see the results live


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059939001


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061597787


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060658961


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f 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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048885851


   > @xushiyan great! I will do this by today. I'm planning to use it on aws glue which unfortunately only offers spark 3.1 today. I know that Hudi documentation says explicitly that the supported version of spark is 3.2, but there is any chance that it will work on 3.1?
   
   hi  @fedsp Here is a multi version PR, you can test it. https://github.com/apache/hudi/pull/4885


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058776372


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r819186894



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelParser.scala
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.{Project, TimeTravelRelation}
+
+class TestTimeTravelParser extends TestHoodieSqlBase {
+  private val parser = spark.sessionState.sqlParser
+
+  test("time travel of timestamp") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF '2019-01-29 00:37:58'")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal("2019-01-29 00:37:58")),
+          None))) {
+        timeTravelPlan1
+      }
+
+      val timeTravelPlan2 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF 1643119574")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal(1643119574)),
+          None))) {
+        timeTravelPlan2
+      }
+    }
+  }
+
+  test("time travel of version") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "VERSION AS OF 'Snapshot123456789'")

Review comment:
       I think there is no need to throw an exception when parsing. It's just fine to handle it in `HoodieAnalysis#timetravelrelation`. This syntax is also supported in spark 3.3.




-- 
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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r819186894



##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelParser.scala
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.{Project, TimeTravelRelation}
+
+class TestTimeTravelParser extends TestHoodieSqlBase {
+  private val parser = spark.sessionState.sqlParser
+
+  test("time travel of timestamp") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF '2019-01-29 00:37:58'")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal("2019-01-29 00:37:58")),
+          None))) {
+        timeTravelPlan1
+      }
+
+      val timeTravelPlan2 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF 1643119574")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal(1643119574)),
+          None))) {
+        timeTravelPlan2
+      }
+    }
+  }
+
+  test("time travel of version") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "VERSION AS OF 'Snapshot123456789'")

Review comment:
       I think there is no need to throw an exception when parsing. It's just fine to handle it in `HoodieAnalysis#timetravelrelation`.




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060524956


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061627859


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059956714


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   * 62a9c1d08032f660586edcf05665ca2e440242b9 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059951217


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   
   <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] YannByron commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820360573



##########
File path: hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/adapter/Spark3_2Adapter.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark3RowSerDe
+import org.apache.hudi.client.utils.SparkRowSerDe
+import org.apache.hudi.spark3.internal.ReflectUtil
+import org.apache.spark.sql.avro.{HoodieAvroDeserializerTrait, HoodieAvroSerializer, HoodieAvroSerializerTrait, Spark3HoodieAvroDeserializer}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.JoinType
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, Spark3ParsePartitionUtil, SparkParsePartitionUtil}
+import org.apache.spark.sql.hudi.SparkAdapter
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.parser.HoodieSpark3_2ExtendedSqlParser
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.{Row, SparkSession}
+
+/**
+ * The adapter for spark3.2.
+ */
+class Spark3_2Adapter extends SparkAdapter {

Review comment:
       can you let Spark3_2Adapter extend Spark3Adapter, and only overwrite `isRelationTimeTravel` and `getRelationTimeTravel`.




-- 
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] YannByron commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820358622



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -396,6 +398,37 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case l if sparkAdapter.isRelationTimeTravel(l) =>
+      val (plan: UnresolvedRelation, timestamp, version) =
+        sparkAdapter.getRelationTimeTravel(l).get
+
+      if (timestamp.isEmpty && version.nonEmpty) {
+        throw new AnalysisException(
+          "version expression is not supported for time travel")
+      }
+
+      val tableIdentifier = sparkAdapter.toTableIdentifier(plan)
+      if (sparkAdapter.isHoodieTable(tableIdentifier, sparkSession)) {
+        val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier)
+        val table = hoodieCatalogTable.table
+        val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_))
+        val instantOption = Map(
+          DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key -> timestamp.get.toString())
+        val dataSource =
+          DataSource(
+            sparkSession,
+            userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema),
+            partitionColumns = table.partitionColumnNames,
+            bucketSpec = table.bucketSpec,
+            className = table.provider.get,
+            options = table.storage.properties ++ pathOption ++ instantOption,
+            catalogTable = Some(table))
+
+        LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table)
+      } else {
+        plan

Review comment:
       if not a Hoodie table, it should return the origin object: `l`.




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060679450


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059933314


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   
   <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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060168443


   Now, from a spark context (glue context), I tried the following pyspark command:
   
   `df = spark_context.read("SELECT * FROM SANDBOX.hudi_0_11_tst TIMESTAMP AS OF '2022-03-06 15:30:58'")`
   
   and it gave me the following error:
   
   ```
   File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/context.py", line 433, in sql
       return self.sparkSession.sql(sqlQuery)
     File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/session.py", line 723, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
     File "/opt/amazon/spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in _call_
       answer, self.gateway_client, self.target_id, self.name)
     File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 117, in deco
       raise converted from None
   pyspark.sql.utils.ParseException: 
   mismatched input 'AS' expecting {<EOF>, ';'}(line 1, pos 46)
   
   == SQL ==
   SELECT * FROM SANDBOX.hudi_0_11_tst TIMESTAMP AS OF '2022-03-06 15:30:58'
   ----------------------------------------------^^^
   ```


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058765558


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060382267


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060592046


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061632285


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 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] xushiyan merged pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan merged pull request #4720:
URL: https://github.com/apache/hudi/pull/4720


   


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1039772392


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   * 7af196e1c42c72816f1709dd982293045523906d 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040106705


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040153696


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041460996


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   * 9efcab8b8f103d07395badb779c875d76234717d 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1034283886


   @xushiyan , there is anything that I can help from my side to help  you on this PR? 
   I can help testing using real datasets


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041372256


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041290326


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   * 9234e97b9428e00d78724ace385200152e9af1f6 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041036311


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   * 47745008de1658561623c4187330e1004794d253 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1039829883


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   
   <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] YannByron commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
YannByron commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1054919424


   > @YannByron can you help reviewing this too? thanks
   
   As discussed with @XuQianJin-Stars, I prefer to make a separate pr based on the separate spark env with https://issues.apache.org/jira/browse/SPARK-37219. And we can merge this to hudi master once Spark3.3 releases.


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025046116


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   
   <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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060099019


   No, @XuQianJin-Stars , unfortunatelly it is 3.1, since I am limited to the aws glue environment. But I used your branch https://github.com/apache/hudi/pull/4885
   


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061785231


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059980634


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060559463


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   * 62a9c1d08032f660586edcf05665ca2e440242b9 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060513787


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061573797


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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] xushiyan commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820245283



##########
File path: hudi-spark-datasource/hudi-spark3/pom.xml
##########
@@ -157,7 +175,7 @@
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_2.12</artifactId>
-      <version>${spark3.version}</version>
+      <version>${spark3.2.version}</version>

Review comment:
       we should prefer to simple set of properties to maintain: both `spark3.version` and `hudi.spark.module` change based on different spark profile. When `spark3.version` switch to `3.1.2`, module will be `hudi-spark3.1.x` and effectively ignores `hudi-spark3` here. So I don't think we need to introduce more properties here.




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025161997


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025752256


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   
   <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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040195608


   hi @fedsp well, you can use this patch to help verify it.


-- 
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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040195608


   hi @fedsp well, you can use this patch to help verify it.


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026960110


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026783306


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059900208


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f 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] fedsp edited a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp edited a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060168443


   Now, from a spark context (glue context), I tried the following pyspark command:
   
   `df = spark_context.sql("SELECT * FROM SANDBOX.hudi_0_11_tst TIMESTAMP AS OF '2022-03-06 15:30:58'")`
   
   and it gave me the following error:
   
   ```
   File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/context.py", line 433, in sql
       return self.sparkSession.sql(sqlQuery)
     File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/session.py", line 723, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
     File "/opt/amazon/spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", line 1305, in _call_
       answer, self.gateway_client, self.target_id, self.name)
     File "/opt/amazon/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 117, in deco
       raise converted from None
   pyspark.sql.utils.ParseException: 
   mismatched input 'AS' expecting {<EOF>, ';'}(line 1, pos 46)
   
   == SQL ==
   SELECT * FROM SANDBOX.hudi_0_11_tst TIMESTAMP AS OF '2022-03-06 15:30:58'
   ----------------------------------------------^^^
   ```


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060703798


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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] YannByron commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
YannByron commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820360093



##########
File path: hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/AnalysisException.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.annotation.Stable
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.Origin
+
+/**
+ * Thrown when a query fails to analyze, usually because the query itself is invalid.
+ *
+ * @since 1.3.0
+ */
+@Stable
+class AnalysisException protected[sql](

Review comment:
       why we need to define this instead of using `AnalysisException` inside of Spark.




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   * 53e78979a5a780834ce4a9bf34a9119544b0054f 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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r818241910



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       > do you have a follow-up ticket for supporting version-only time-travel?
   
   I have no plan for this one at present.




-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061787731


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026650659


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) 
   
   <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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026781441


   @hudi-bot run azure


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025755179


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   * bd1cd8e855a73184363cfdb94560480299b34c72 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   * 47745008de1658561623c4187330e1004794d253 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059966301


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) 
   
   <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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060678330


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059938566


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059957128


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596) 
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) 
   
   <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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059968536


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060439519


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   
   <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] xushiyan commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r819386591



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       > > do you have a follow-up ticket for supporting version-only time-travel?
   > 
   > I have no plan for this one at present. I'll look at the implementation based on version time travel later
   
   @XuQianJin-Stars since we just need to implement instant as of, then there's no strong requirement to support `version` 




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061576317


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c 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] xushiyan edited a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan edited a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059884088


   > @xushiyan great! I will do this by today. I'm planning to use it on aws glue which unfortunately only offers spark 3.1 today. I know that Hudi documentation says explicitly that the supported version of spark is 3.2, but there is any chance that it will work on 3.1?
   
   This PR is ready for testing with `TIMESTAMP AS OF` syntax to fetch older commits' values. @fedsp any chance you have tested this branch out? we'd happy to land this with some real-world verifications. :)


-- 
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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059968536


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060652625


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060374073


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r819545642



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       > > > do you have a follow-up ticket for supporting version-only time-travel?
   > > 
   > > 
   > > I have no plan for this one at present. I'll look at the implementation based on version time travel later
   > 
   > @XuQianJin-Stars since we just need to implement instant as of, then there's no strong requirement to support `version`
   
   well




-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060682232


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1027032434


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1027032434






-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040103750


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061720767


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6692) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 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] xushiyan commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r817782917



##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
##########
@@ -27,7 +27,9 @@ import org.apache.spark.sql.hudi.SparkAdapter
 trait SparkAdapterSupport {
 
   lazy val sparkAdapter: SparkAdapter = {
-    val adapterClass = if (HoodieSparkUtils.isSpark3) {
+    val adapterClass = if (HoodieSparkUtils.isSpark3_2) {

Review comment:
       use `gteqSpark3_2` here to be future-proof ?

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala
##########
@@ -27,7 +27,9 @@ import org.apache.spark.sql.hudi.SparkAdapter
 trait SparkAdapterSupport {
 
   lazy val sparkAdapter: SparkAdapter = {
-    val adapterClass = if (HoodieSparkUtils.isSpark3) {
+    val adapterClass = if (HoodieSparkUtils.isSpark3_2) {
+      "org.apache.spark.sql.adapter.Spark3_2Adapter"
+    } else if (HoodieSparkUtils.isSpark3 || HoodieSparkUtils.isSpark3_1) {

Review comment:
       we should deprecate `isSpark3` since it's ambiguous. 

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelParser.scala
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.{Project, TimeTravelRelation}
+
+class TestTimeTravelParser extends TestHoodieSqlBase {
+  private val parser = spark.sessionState.sqlParser
+
+  test("time travel of timestamp") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF '2019-01-29 00:37:58'")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal("2019-01-29 00:37:58")),
+          None))) {
+        timeTravelPlan1
+      }
+
+      val timeTravelPlan2 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF 1643119574")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal(1643119574)),
+          None))) {
+        timeTravelPlan2
+      }
+    }
+  }
+
+  test("time travel of version") {
+    if (HoodieSparkUtils.isSpark3_2) {

Review comment:
       ditto

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelParser.scala
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.{Project, TimeTravelRelation}
+
+class TestTimeTravelParser extends TestHoodieSqlBase {
+  private val parser = spark.sessionState.sqlParser
+
+  test("time travel of timestamp") {
+    if (HoodieSparkUtils.isSpark3_2) {

Review comment:
       better negate the checking and early return 

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?
+      if (timestamp.isEmpty && version.nonEmpty) {
+        throw new AnalysisException(
+          "version expression is not support for time travel")

Review comment:
       ```suggestion
             "version expression is not supported for time travel")
   ```

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       2nd thoughts: we should keep the usage similar to what has been done here https://hudi.apache.org/docs/quick-start-guide/#time-travel-query
   
   so just support `INSTANT AS OF ` and parse those 3 time formats?

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.hudi.common.table.HoodieTableMetaClient
+
+class TestTimeTravelTable extends TestHoodieSqlBase {
+
+  test("Test Two Table's Union Join with time travel") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      withTempDir { tmp =>
+        Seq("cow", "mor").foreach { tableType =>
+          val tableName = generateTableName
+
+          val basePath = tmp.getCanonicalPath
+          val tableName1 = tableName + "_1"
+          val tableName2 = tableName + "_2"
+          val path1 = s"$basePath/$tableName1"
+          val path2 = s"$basePath/$tableName2"
+
+          spark.sql(
+            s"""
+               |create table $tableName1 (
+               |  id int,
+               |  name string,
+               |  price double,
+               |  ts long
+               |) using hudi
+               | tblproperties (
+               |  type = '$tableType',
+               |  primaryKey = 'id',
+               |  preCombineField = 'ts'
+               | )
+               | location '$path1'
+       """.stripMargin)
+
+          spark.sql(
+            s"""
+               |create table $tableName2 (
+               |  id int,
+               |  name string,
+               |  price double,
+               |  ts long
+               |) using hudi
+               | tblproperties (
+               |  type = '$tableType',
+               |  primaryKey = 'id',
+               |  preCombineField = 'ts'
+               | )
+               | location '$path2'
+       """.stripMargin)
+
+          spark.sql(s"insert into $tableName1 values(1, 'a1', 10, 1000)")
+          spark.sql(s"insert into $tableName1 values(2, 'a2', 20, 1000)")
+
+          checkAnswer(s"select id, name, price, ts from $tableName1")(
+            Seq(1, "a1", 10.0, 1000),
+            Seq(2, "a2", 20.0, 1000)
+          )
+
+          checkAnswer(s"select id, name, price, ts from $tableName1")(
+            Seq(1, "a1", 10.0, 1000),
+            Seq(2, "a2", 20.0, 1000)
+          )
+
+          spark.sql(s"insert into $tableName2 values(3, 'a3', 10, 1000)")
+          spark.sql(s"insert into $tableName2 values(4, 'a4', 20, 1000)")
+
+          checkAnswer(s"select id, name, price, ts from $tableName2")(
+            Seq(3, "a3", 10.0, 1000),
+            Seq(4, "a4", 20.0, 1000)
+          )
+
+          val metaClient1 = HoodieTableMetaClient.builder()
+            .setBasePath(path1)
+            .setConf(spark.sessionState.newHadoopConf())
+            .build()
+
+          val metaClient2 = HoodieTableMetaClient.builder()
+            .setBasePath(path2)
+            .setConf(spark.sessionState.newHadoopConf())
+            .build()
+
+          val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline
+            .lastInstant().get().getTimestamp
+
+          val instant2 = metaClient2.getActiveTimeline.getAllCommitsTimeline
+            .lastInstant().get().getTimestamp
+
+          val sql =
+            s"""
+               |select id, name, price, ts from $tableName1 TIMESTAMP AS OF '$instant1' where id=1
+               |union
+               |select id, name, price, ts from $tableName2 TIMESTAMP AS OF '$instant2' where id>1
+               |""".stripMargin

Review comment:
       not sure how exactly this union query can test time travel functionality effectively. if nothing particular about union here, then we should rewrite the test for time travel specifically. I think we need to insert at commit1 and update the same record at commit2. Then use as of to query and get the value at commit1's time.

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       do you have a follow-up ticket for supporting version-only time-travel?

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelParser.scala
##########
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.logical.{Project, TimeTravelRelation}
+
+class TestTimeTravelParser extends TestHoodieSqlBase {
+  private val parser = spark.sessionState.sqlParser
+
+  test("time travel of timestamp") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF '2019-01-29 00:37:58'")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal("2019-01-29 00:37:58")),
+          None))) {
+        timeTravelPlan1
+      }
+
+      val timeTravelPlan2 = parser.parsePlan("SELECT * FROM A.B " +
+        "TIMESTAMP AS OF 1643119574")
+
+      assertResult(Project(Seq(UnresolvedStar(None)),
+        TimeTravelRelation(
+          UnresolvedRelation(new TableIdentifier("B", Option.apply("A"))),
+          Some(Literal(1643119574)),
+          None))) {
+        timeTravelPlan2
+      }
+    }
+  }
+
+  test("time travel of version") {
+    if (HoodieSparkUtils.isSpark3_2) {
+      val timeTravelPlan1 = parser.parsePlan("SELECT * FROM A.B " +
+        "VERSION AS OF 'Snapshot123456789'")

Review comment:
       shouldn't this throw exception as not supported?

##########
File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestTimeTravelTable.scala
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi
+
+import org.apache.hudi.HoodieSparkUtils
+import org.apache.hudi.common.table.HoodieTableMetaClient
+
+class TestTimeTravelTable extends TestHoodieSqlBase {
+
+  test("Test Two Table's Union Join with time travel") {
+    if (HoodieSparkUtils.isSpark3_2) {

Review comment:
       same here. gteqSpark3_2 should work better. also better early return




-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058833127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025046503


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026647826


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647) 
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041368743


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   * 9234e97b9428e00d78724ace385200152e9af1f6 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   
   <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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r818241910



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieAnalysis.scala
##########
@@ -350,6 +353,35 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi
         l
       }
 
+    case TimeTravelRelation(plan: UnresolvedRelation, timestamp, version) =>
+      // TODO: How to use version to perform time travel?

Review comment:
       > do you have a follow-up ticket for supporting version-only time-travel?
   
   I have no plan for this one at present.  I'll look at the implementation based on version time travel later




-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061849309


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041532023


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9efcab8b8f103d07395badb779c875d76234717d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058767066


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058793982


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) 
   
   <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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060018898


   Hi @xushiyan! Sorry for the long response, I had some trouble building the .jar (not used to do it)
   
   Anyways, I'm receiving a error from a straight sql statement running from AwsAthena (no problem creating the table on aws glue using spark 3 tho)
   
   This is the error:
   ![image](https://user-images.githubusercontent.com/22076750/156937853-50cf73b1-85dc-429b-a3e5-05dd037027da.png)
   
   And this is the query that I used on athena:
   `select * from hudi_0_11_tst TIMESTAMP AS OF '2022-03-06 15:30:58'`
   
   and this is the DDL of the table creation (I created the table by hand, not on the spark write operation):
   ```
   CREATE EXTERNAL TABLE hudi_0_11_tst(
       `tpep_pickup_datetime` timestamp,
       `tpep_dropoff_datetime` timestamp,
       `passenger_count` int,
       `trip_distance` double,
       `ratecodeid` int,
       `store_and_fwd_flag` string,
       `pulocationid` int,
       `dolocationid` int,
       `payment_type` int,
       `fare_amount` double,
       `extra` double,
       `mta_tax` double,
       `tip_amount` double,
       `tolls_amount` double,
       `improvement_surcharge` double,
       `total_amount` double,
       `congestion_surcharge` double,
       `pk_col` bigint
   )
   PARTITIONED BY (
       `vendorid` string)
   ROW FORMAT SERDE
       'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
   WITH SERDEPROPERTIES ( 
       'path'='s3://mybuckethudi_0_11_tst')
   STORED AS INPUTFORMAT
       'org.apache.hudi.hadoop.HoodieParquetInputFormat'
   OUTPUTFORMAT
       'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
   LOCATION
       's3://mybucket/hudi_0_11_tst'
   ```
   
   
   Did I did something wrong?


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060379684


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060519440


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1b0ff01075418a5347edc08fc078e5dad80ba99b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632) 
   * 869a752768dabc9e9cdb7df6b725fba50dd502af 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060596636


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 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] XuQianJin-Stars commented on a change in pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on a change in pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#discussion_r820180708



##########
File path: hudi-spark-datasource/hudi-spark3/pom.xml
##########
@@ -157,7 +175,7 @@
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_2.12</artifactId>
-      <version>${spark3.version}</version>
+      <version>${spark3.2.version}</version>

Review comment:
       > can you clarify why this change? what if user needs to build the project with spark 3.1.x profile?
   
   spark 3.1.x  profile use `<hudi.spark.module>hudi-spark3.1.x</hudi.spark.module>` with spark3.1.x version.
   spark 3  profile use `<hudi.spark.module>hudi-spark3</hudi.spark.module>` with spark 3.2.0(and above) versions.




-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059899983


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525) 
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060371083


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641) 
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   * 7af196e1c42c72816f1709dd982293045523906d 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040156614


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040298978


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041068934


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041430396


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9234e97b9428e00d78724ace385200152e9af1f6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057) 
   
   <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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040220274


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1039774211


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3438e30d235fbdaa9973ebbb2ce2ccfe64124a34 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652) 
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040222786


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027) 
   
   <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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040195608


   hi @fedsp well, you can use this patch to help verify it.


-- 
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] XuQianJin-Stars commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1040220274


   @hudi-bot run azure


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7af196e1c42c72816f1709dd982293045523906d Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011) 
   * 15931c0e37361afd9732f04305b8ab48ce305aac Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026) 
   * c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e0fadab437970eac2b15816d101db50586bc7d9b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1041287810


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 47745008de1658561623c4187330e1004794d253 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040) 
   * e0fadab437970eac2b15816d101db50586bc7d9b 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 53e78979a5a780834ce4a9bf34a9119544b0054f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242) 
   
   <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] fedsp commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
fedsp commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1048755045


   @xushiyan great! I will do this by today. I'm planning to use it on aws glue which unfortunately only offers spark 3.1 today. I know that Hudi documentation says explicitly that the supported version of spark is 3.2, but there is any chance that it will work on 3.1?


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026715785


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646) 
   
   <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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   * 6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025868520


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4e6424b7b34d1f79972b7932f5e33eb0c4f18005 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621) 
   
   <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] XuQianJin-Stars removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1026781441


   @hudi-bot run azure


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025161332


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   * bd1cd8e855a73184363cfdb94560480299b34c72 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025055058


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 21448f9a99fdc61cb594ad183690650f27a5e6d4 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1025173797


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bd1cd8e855a73184363cfdb94560480299b34c72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060630659


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 8595490aee9edbbf358a964e485fbd53bb4af073 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635) 
   * a065ed76da7d25227c708324309f2845ca7162f3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061593311


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6632",
       "triggerID" : "1b0ff01075418a5347edc08fc078e5dad80ba99b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "869a752768dabc9e9cdb7df6b725fba50dd502af",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6635",
       "triggerID" : "8595490aee9edbbf358a964e485fbd53bb4af073",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6638",
       "triggerID" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a065ed76da7d25227c708324309f2845ca7162f3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6641",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "59e0270ef6aba60d4a80f2482ec3fa605cd01e1f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642",
       "triggerID" : "1060678330",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b989ca6df7345d60930ede5628de28b3caae5147",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8b62dd5cad44ab2cd81f12206da683cf139c574c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 869a752768dabc9e9cdb7df6b725fba50dd502af UNKNOWN
   * 59e0270ef6aba60d4a80f2482ec3fa605cd01e1f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6642) 
   * b989ca6df7345d60930ede5628de28b3caae5147 UNKNOWN
   * 8b62dd5cad44ab2cd81f12206da683cf139c574c UNKNOWN
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 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] xushiyan commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1062053812


   @fedsp it's a spark bundle or version mismatch problem where the syntax is not recognized. Maybe previous version from the branch has some misconfig but now it's resolved. I verified the feature in spark 3.2.1
   
   ```
   ➜ mvn -T 2.5C clean install -DskipTests -Djacoco.skip=true -Dmaven.javadoc.skip=true -Dcheckstyle.skip=true -Dscala-2.12 -Dspark3
   
   
   // Spark 3.2.1
   // COW
   spark.sql("""
   create table hudi_cow_pt_tbl (
     id bigint,
     name string,
     ts bigint,
     dt string,
     hh string
   ) using hudi
   tblproperties (
     type = 'cow',
     primaryKey = 'id',
     preCombineField = 'ts'
    )
   partitioned by (dt, hh)
   location '/tmp/hudi/hudi_cow_pt_tbl';
   """)
   spark.sql("insert into hudi_cow_pt_tbl select 1, 'a0', 1000, '2021-12-09', '10'")
   spark.sql("select * from hudi_cow_pt_tbl").show()
   spark.sql("insert into hudi_cow_pt_tbl select 1, 'a1', 1001, '2021-12-09', '10'")
   spark.sql("select * from hudi_cow_pt_tbl").show()
   
   
   // time travel based on first commit time
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '20220308175415995' where id = 1").show()
   // time travel based on different timestamp formats
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '2022-03-08 17:54:15.995' where id = 1").show()
   spark.sql("select * from hudi_cow_pt_tbl timestamp as of '2022-03-09' where id = 1").show()
   ```
   
   


-- 
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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627",
       "triggerID" : "dde240c645b629714f27ecfed5d50c1140cba046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   * dde240c645b629714f27ecfed5d50c1140cba046 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6627) 
   
   <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] xushiyan commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1060133169


   @fedsp let me clarify that the time travel query here is supported in Spark SQL with Spark 3.2+ but not yet in other query engine like presto. So you won't be able to run this in Athena. Are you able to verify the branch by running spark sql against your datasets?


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061846321


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061790085


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   
   <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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1061852406


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "1061627859",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699",
       "triggerID" : "3cc3b11a9533b1c388bb257c09a9344e1b062185",
       "triggerType" : "PUSH"
     }, {
       "hash" : "82a888652bf47775fe2626c259129a367b38f766",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701",
       "triggerID" : "82a888652bf47775fe2626c259129a367b38f766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3cc3b11a9533b1c388bb257c09a9344e1b062185 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6695) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6699) 
   * 82a888652bf47775fe2626c259129a367b38f766 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6701) 
   
   <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] xushiyan commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059884088


   > @xushiyan great! I will do this by today. I'm planning to use it on aws glue which unfortunately only offers spark 3.1 today. I know that Hudi documentation says explicitly that the supported version of spark is 3.2, but there is any chance that it will work on 3.1?
   
   @fedsp any chance you have tested this branch out? we'd happy to land this with some real-world verifications. :)


-- 
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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1059932873


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586) 
   * 9475a59cc9d809670f4b1f607f3fd9c562fe3033 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594) 
   * 908fec5565135512f1bfeb2e23bac79a5bf898c3 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 removed a comment on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1058777588


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e54348357a6fae653f3e6fa35e93368bdf125f6e Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522) 
   * b1e39e413a9fc660f50dd5e8176200f64ec29b28 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 #4720: [HUDI-3221] Support querying a table as of a savepoint

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5611",
       "triggerID" : "21448f9a99fdc61cb594ad183690650f27a5e6d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5616",
       "triggerID" : "bd1cd8e855a73184363cfdb94560480299b34c72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5621",
       "triggerID" : "4e6424b7b34d1f79972b7932f5e33eb0c4f18005",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5646",
       "triggerID" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f75b5d693bcc5a4bedfb7cd6fd08d974d1355b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5647",
       "triggerID" : "1026781441",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5652",
       "triggerID" : "3438e30d235fbdaa9973ebbb2ce2ccfe64124a34",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7af196e1c42c72816f1709dd982293045523906d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6011",
       "triggerID" : "7af196e1c42c72816f1709dd982293045523906d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6026",
       "triggerID" : "15931c0e37361afd9732f04305b8ab48ce305aac",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6029",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c9b323fc59f63b23a616d2bcf5880a5c3b7d91f2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6027",
       "triggerID" : "1040220274",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "47745008de1658561623c4187330e1004794d253",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6040",
       "triggerID" : "47745008de1658561623c4187330e1004794d253",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6055",
       "triggerID" : "e0fadab437970eac2b15816d101db50586bc7d9b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6057",
       "triggerID" : "9234e97b9428e00d78724ace385200152e9af1f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9efcab8b8f103d07395badb779c875d76234717d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6060",
       "triggerID" : "9efcab8b8f103d07395badb779c875d76234717d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6242",
       "triggerID" : "53e78979a5a780834ce4a9bf34a9119544b0054f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6522",
       "triggerID" : "e54348357a6fae653f3e6fa35e93368bdf125f6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6525",
       "triggerID" : "b1e39e413a9fc660f50dd5e8176200f64ec29b28",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6586",
       "triggerID" : "e59eeceea47b6c6b0a3c0310b1abe2ac7e54780a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6594",
       "triggerID" : "9475a59cc9d809670f4b1f607f3fd9c562fe3033",
       "triggerType" : "PUSH"
     }, {
       "hash" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6596",
       "triggerID" : "908fec5565135512f1bfeb2e23bac79a5bf898c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598",
       "triggerID" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "62a9c1d08032f660586edcf05665ca2e440242b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602",
       "triggerID" : "1059968536",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 62a9c1d08032f660586edcf05665ca2e440242b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6598) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=6602) 
   
   <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] xushiyan commented on pull request #4720: [HUDI-3221] Support querying a table as of a savepoint

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #4720:
URL: https://github.com/apache/hudi/pull/4720#issuecomment-1054330603


   @YannByron can you help reviewing this too? thanks


-- 
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