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/14 17:45:16 UTC

[GitHub] [hudi] nsivabalan opened a new pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   ## What is the purpose of the pull request
   
   Introducing restore.requested instant for restore action. 
   - Restore is split into restore plan and restore execution phase. 
   - A restore Plan is added which gets serialized into restore.requested instant during planning phase. This contains the list of instants to rollback. 
   - During execution phase, restore plan is read from the requested instant and individual rollbacks are executed. 
   - Once completed, timeline will contain 3 meta files for restore (requested, inflight and complete) and no meta files for rollbacks will be present in timeline. 
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   * b1ec8055e09845d859b31491e91d4087b5ca172a 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821",
       "triggerID" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   * 4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   * 4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50 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] nsivabalan commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.table.action.restore;
+
+import org.apache.hudi.avro.model.HoodieRestorePlan;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+
+import java.io.IOException;
+
+public class RestoreUtils {
+
+  /**
+   * Get Latest version of Restore plan corresponding to a restore instant.
+   *
+   * @param metaClient      Hoodie Table Meta Client
+   * @param restoreInstant Instant referring to restore action
+   * @return Rollback plan corresponding to rollback instant
+   * @throws IOException
+   */
+  public static HoodieRestorePlan getRestorePlan(HoodieTableMetaClient metaClient, HoodieInstant restoreInstant)
+      throws IOException {
+    // TODO: add upgrade step if required.

Review comment:
       yes. but here is what I am thinking. since restore has to be done via hudi-cli anyways, may be we should not recommend upgrading a hudi table with a pending restore. let the user ensure restore is complete. or can delete the restore instants and retry after upgrade is complete. lets jam on this to get a consensus. Can take it as a follow up. 




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

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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -674,16 +675,21 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H
     Timer.Context timerContext = metrics.getRollbackCtx();
     try {
       HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
-      HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
-      if (timerContext != null) {
-        final long durationInMs = metrics.getDurationInMs(timerContext.stop());
-        final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
-            .flatMap(Collection::stream)
-            .mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
-            .sum();
-        metrics.updateRollbackMetrics(durationInMs, totalFilesDeleted);
+      Option<HoodieRestorePlan> restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime);

Review comment:
       nope. we don't guarantee atomicity. In general, restore is a destructive operation and its recommended to stop all writers before doing a restore for any given hudi table. Even queries might fail if reading into files which restore might clean it up, since restore will do eager cleaning of files from latest commits.  
   
   Given this state, can you go over your question again? Concurrent restores is not advisable and might end up in an inconsistent state. So, are you asking about two subsequent restores? 
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   * 4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   
   <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] nsivabalan commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -348,6 +349,13 @@ public HoodieTimeline getRollbackTimeline() {
     return getActiveTimeline().getRollbackTimeline();
   }
 
+  /**
+   * Get restore timeline.
+   */
+  public HoodieTimeline getRestoreTimeline() {

Review comment:
       we have a precedence and I am just following the same 
   ```
   public HoodieTimeline getCleanTimeline() {
       return getActiveTimeline().getCleanerTimeline();
     }
   
     /**
      * Get rollback timeline.
      */
     public HoodieTimeline getRollbackTimeline() {
       return getActiveTimeline().getRollbackTimeline();
     }
   
     /**
      * Get restore timeline.
      */
     public HoodieTimeline getRestoreTimeline() {
       return getActiveTimeline().getRestoreTimeline();
     }
   .
   .
   
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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] codope commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -348,6 +349,13 @@ public HoodieTimeline getRollbackTimeline() {
     return getActiveTimeline().getRollbackTimeline();
   }
 
+  /**
+   * Get restore timeline.
+   */
+  public HoodieTimeline getRestoreTimeline() {

Review comment:
       Is there a need for this API? Can we not do `table.getActiveTimeline().getRestoreTimeline()` in the action executor?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
##########
@@ -391,6 +396,20 @@ public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant request
     return inflight;
   }
 
+  /**
+   * Transition Restore State from requested to inflight.
+   *
+   * @param requestedInstant requested instant
+   * @return commit instant
+   */
+  public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requestedInstant) {
+    ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.RESTORE_ACTION));

Review comment:
       Let's add meanigful message for validation failures.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -674,16 +675,21 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H
     Timer.Context timerContext = metrics.getRollbackCtx();
     try {
       HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
-      HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
-      if (timerContext != null) {
-        final long durationInMs = metrics.getDurationInMs(timerContext.stop());
-        final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
-            .flatMap(Collection::stream)
-            .mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
-            .sum();
-        metrics.updateRollbackMetrics(durationInMs, totalFilesDeleted);
+      Option<HoodieRestorePlan> restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime);

Review comment:
       If i understand correctly, then the assumption is still that no other writer is inflight when restore is going to be scheduled or executed, and that the restore is atomic i.e. all or none of the instants in the restore plan are rolled back?
   Just curious, what happens if user schedules another restore while one was in progress? Will it even generate a new plan  given that no writes were committed in between? If the plan will be generated then can it go inflight when another is in progress?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java
##########
@@ -353,6 +354,10 @@ static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) {
     return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant);
   }
 
+  static HoodieInstant getRestoreRequestedInstant(HoodieInstant instant) {

Review comment:
       Where is this being used?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
##########
@@ -65,29 +71,53 @@ public HoodieRestoreMetadata execute() {
     HoodieTimer restoreTimer = new HoodieTimer();
     restoreTimer.startTimer();
 
-    // Get all the commits on the timeline after the provided commit time
-    List<HoodieInstant> instantsToRollback = table.getActiveTimeline().getWriteTimeline()
-        .getReverseOrderedInstants()
-        .filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), restoreInstantTime))
-        .collect(Collectors.toList());
+    Option<HoodieInstant> restoreInstant = table.getRestoreTimeline()
+        .filterInflightsAndRequested()
+        .filter(instant -> instant.getTimestamp().equals(instantTime))
+        .firstInstant();
+    if (!restoreInstant.isPresent()) {
+      throw new HoodieRollbackException("No pending restore instants found to execute restore");

Review comment:
       Would `instantTime` and `restoreInstantTime` in the execption message be helpful?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.table.action.restore;
+
+import org.apache.hudi.avro.model.HoodieRestorePlan;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+
+import java.io.IOException;
+
+public class RestoreUtils {
+
+  /**
+   * Get Latest version of Restore plan corresponding to a restore instant.
+   *
+   * @param metaClient      Hoodie Table Meta Client
+   * @param restoreInstant Instant referring to restore action
+   * @return Rollback plan corresponding to rollback instant
+   * @throws IOException
+   */
+  public static HoodieRestorePlan getRestorePlan(HoodieTableMetaClient metaClient, HoodieInstant restoreInstant)
+      throws IOException {
+    // TODO: add upgrade step if required.

Review comment:
       This upgrade step would only be required if users upgrade Hudi while there is a pending restore in timeline. Is my understanding correct?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
##########
@@ -99,12 +129,23 @@ private HoodieRestoreMetadata finishRestore(Map<String, List<HoodieRollbackMetad
     writeToMetadata(restoreMetadata);
     table.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, instantTime),
         TimelineMetadataUtils.serializeRestoreMetadata(restoreMetadata));
+    // get all rollbacks instants after restore instant time and delete them.

Review comment:
       change comment to be more clear: "get all **pending** rollbacks instants..."




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   
   <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] codope commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -674,16 +675,21 @@ public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws H
     Timer.Context timerContext = metrics.getRollbackCtx();
     try {
       HoodieTable<T, I, K, O> table = createTable(config, hadoopConf, config.isMetadataTableEnabled());
-      HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
-      if (timerContext != null) {
-        final long durationInMs = metrics.getDurationInMs(timerContext.stop());
-        final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
-            .flatMap(Collection::stream)
-            .mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
-            .sum();
-        metrics.updateRollbackMetrics(durationInMs, totalFilesDeleted);
+      Option<HoodieRestorePlan> restorePlanOption = table.scheduleRestore(context, restoreInstantTime, instantTime);

Review comment:
       Yes, I was talking about two subsequent restores (with the same restoreToInstant) in which one is inflight and the other is requested.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   * b1ec8055e09845d859b31491e91d4087b5ca172a 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   
   <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] nsivabalan merged pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821",
       "triggerID" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   * 4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   * b1ec8055e09845d859b31491e91d4087b5ca172a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff33ac79d17e132f24ed707c200409f23655c548 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391) 
   
   <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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   * ff33ac79d17e132f24ed707c200409f23655c548 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253) 
   * ff33ac79d17e132f24ed707c200409f23655c548 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] nsivabalan commented on a change in pull request #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java
##########
@@ -353,6 +354,10 @@ static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) {
     return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant);
   }
 
+  static HoodieInstant getRestoreRequestedInstant(HoodieInstant instant) {

Review comment:
       looks like not required with current state. may be added it initially. will remove 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd 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 #4605: [HUDI-2432] Adding restore.requested instant and restore plan for restore action

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5253",
       "triggerID" : "2b1f80fe73e4a53266ccd8931bb244cbf7a1b4cd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5391",
       "triggerID" : "ff33ac79d17e132f24ed707c200409f23655c548",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5427",
       "triggerID" : "b1ec8055e09845d859b31491e91d4087b5ca172a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821",
       "triggerID" : "4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4ad83d6bb3f53d539a4dfeef53d3c2f9ae065e50 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=5821) 
   
   <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