You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/01/27 12:45:16 UTC

[GitHub] [flink] dawidwys opened a new pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

dawidwys opened a new pull request #18539:
URL: https://github.com/apache/flink/pull/18539


   ## What is the purpose of the change
   
   Incremental savepoints do not reuse sst files from previous checkpoints.
   At this point they re-upload those files. Moreover they do not register
   its files as reusable by future checkpoints. Lastly, all sst files are
   created in the EXCLUSIVE scope with relative paths, which makes the
   savepoint relocatable.
   
   In order to support CLAIM mode for such savepoints, if a CLAIMed
   snapshots contain shared files, we delay deleting the exclusive
   directory until shared files coming from that snapshot are not used
   anymore.
   
   ## Verifying this change
   
   Added tests:
   * SavepointFormatITCase
   * SharedStateRegistryTest
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (**yes** / no / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / docs / **JavaDocs** / not documented)
   


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] dawidwys commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r798699282



##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java
##########
@@ -111,14 +111,35 @@
                                 keyedState ->
                                         assertThat(
                                                 keyedState,
-                                                instanceOf(KeyGroupsStateHandle.class))));
+                                                instanceOf(KeyGroupsStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.CANONICAL,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(SavepointKeyedStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.NATIVE,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(
+                                                        IncrementalRemoteKeyedStateHandle.class))));

Review comment:
       It is enough to verify the format of the savepoint. We verify savepoints are working if we restore from the savepoint without exceptions (after relocating 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 528758ad9508f9fec5630b747fe9f454fbc30171 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] pnowojski commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r795622785



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
##########
@@ -203,10 +204,37 @@ public long getStateSize() {
      *
      * @param sharedStateRegistry The registry where shared states are registered
      */
-    public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) {
-        // in claim mode we should not register any shared handles
+    public void registerSharedStatesAfterRestored(
+            SharedStateRegistry sharedStateRegistry, boolean claim) {

Review comment:
       nit: missing java doc

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
##########
@@ -203,10 +204,37 @@ public long getStateSize() {
      *
      * @param sharedStateRegistry The registry where shared states are registered
      */
-    public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) {
-        // in claim mode we should not register any shared handles
+    public void registerSharedStatesAfterRestored(
+            SharedStateRegistry sharedStateRegistry, boolean claim) {
+        // in no_claim mode we should not register any shared handles
         if (!props.isUnclaimed()) {
-            sharedStateRegistry.registerAll(operatorStates.values(), checkpointID);
+            if (claim) {

Review comment:
       ```
   if (!props.isUnclaimed) { // == if (props.isClaimed())
     if (claim) { // == is doubly claimed?
     } 
     else {  // == it is claimed, but not really?
     }
   }
   ```
   🤔 Can you explain what's happening here?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
##########
@@ -71,5 +71,22 @@ StreamStateHandle registerReference(
      */
     void registerAll(Iterable<? extends CompositeStateHandle> stateHandles, long checkpointID);
 
+    /**
+     * Register given shared states in the registry along with a custom location for shared files.
+     * The custom location will be cleaned, once all of corresponding registered shared handles are
+     * unregistered. Should be used

Review comment:
       > Should be used
   
   ?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java
##########
@@ -224,19 +289,27 @@ public void close() {
         }
     }
 
+    @FunctionalInterface
+    private interface PostDispose {
+        void execute() throws Exception;
+    }
+
     /** Encapsulates the operation the delete state handles asynchronously. */
     private static final class AsyncDisposalRunnable implements Runnable {
 
-        private final StateObject toDispose;
+        private final StreamStateHandle toDispose;
+        private final PostDispose postDispose;
 
-        public AsyncDisposalRunnable(StateObject toDispose) {
+        public AsyncDisposalRunnable(StreamStateHandle toDispose, PostDispose postDispose) {

Review comment:
       Can not we just pass `stateEntry.registryKey` here as a parameter instead of creating a functional interface with lambda functions?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java
##########
@@ -71,5 +71,22 @@ StreamStateHandle registerReference(
      */
     void registerAll(Iterable<? extends CompositeStateHandle> stateHandles, long checkpointID);
 
+    /**
+     * Register given shared states in the registry along with a custom location for shared files.
+     * The custom location will be cleaned, once all of corresponding registered shared handles are
+     * unregistered. Should be used

Review comment:
       nit:
   > Should be used
   
   ?




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   * ed5d5d894e019177d00af8193653c69c21ccc725 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5886c2da10de2ef3e31c94b4bb56e1aceb6deceb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501) 
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] dawidwys commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r795706462



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
##########
@@ -203,10 +204,37 @@ public long getStateSize() {
      *
      * @param sharedStateRegistry The registry where shared states are registered
      */
-    public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) {
-        // in claim mode we should not register any shared handles
+    public void registerSharedStatesAfterRestored(
+            SharedStateRegistry sharedStateRegistry, boolean claim) {
+        // in no_claim mode we should not register any shared handles
         if (!props.isUnclaimed()) {
-            sharedStateRegistry.registerAll(operatorStates.values(), checkpointID);
+            if (claim) {

Review comment:
       `Unclaimed` = `NO_CLAIM` + `LEGACY`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java
##########
@@ -203,10 +204,37 @@ public long getStateSize() {
      *
      * @param sharedStateRegistry The registry where shared states are registered
      */
-    public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) {
-        // in claim mode we should not register any shared handles
+    public void registerSharedStatesAfterRestored(
+            SharedStateRegistry sharedStateRegistry, boolean claim) {
+        // in no_claim mode we should not register any shared handles
         if (!props.isUnclaimed()) {
-            sharedStateRegistry.registerAll(operatorStates.values(), checkpointID);
+            if (claim) {

Review comment:
       `Unclaimed` = `CLAIM` + `LEGACY`




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] dawidwys commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r795709706



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java
##########
@@ -224,19 +289,27 @@ public void close() {
         }
     }
 
+    @FunctionalInterface
+    private interface PostDispose {
+        void execute() throws Exception;
+    }
+
     /** Encapsulates the operation the delete state handles asynchronously. */
     private static final class AsyncDisposalRunnable implements Runnable {
 
-        private final StateObject toDispose;
+        private final StreamStateHandle toDispose;
+        private final PostDispose postDispose;
 
-        public AsyncDisposalRunnable(StateObject toDispose) {
+        public AsyncDisposalRunnable(StreamStateHandle toDispose, PostDispose postDispose) {

Review comment:
       `registryKey` is unfortunately not enough. 
   
   When `AsyncDisposalRunnable` is created from `registerReference` we should not remove the key from the `restoredSharedStates`, as there it is used for simply discarding a duplicated file that is not tracked.




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 528758ad9508f9fec5630b747fe9f454fbc30171 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310) 
   * 5886c2da10de2ef3e31c94b4bb56e1aceb6deceb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] dawidwys commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r798699282



##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java
##########
@@ -111,14 +111,35 @@
                                 keyedState ->
                                         assertThat(
                                                 keyedState,
-                                                instanceOf(KeyGroupsStateHandle.class))));
+                                                instanceOf(KeyGroupsStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.CANONICAL,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(SavepointKeyedStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.NATIVE,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(
+                                                        IncrementalRemoteKeyedStateHandle.class))));

Review comment:
       It is enough to verify the format of the savepoint. We verify savepoints are working if we restore from the savepoint without exceptions.




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] dawidwys merged pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
dawidwys merged pull request #18539:
URL: https://github.com/apache/flink/pull/18539


   


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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot commented on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023171961


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 528758ad9508f9fec5630b747fe9f454fbc30171 (Thu Jan 27 12:48:25 UTC 2022)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   * ed5d5d894e019177d00af8193653c69c21ccc725 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688",
       "triggerID" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   * ed5d5d894e019177d00af8193653c69c21ccc725 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679) 
   * e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] pnowojski commented on a change in pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #18539:
URL: https://github.com/apache/flink/pull/18539#discussion_r798600159



##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java
##########
@@ -111,14 +111,35 @@
                                 keyedState ->
                                         assertThat(
                                                 keyedState,
-                                                instanceOf(KeyGroupsStateHandle.class))));
+                                                instanceOf(KeyGroupsStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.CANONICAL,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(SavepointKeyedStateHandle.class))),
+                Arguments.of(
+                        SavepointFormatType.NATIVE,
+                        ROCKSDB_INCREMENTAL_SNAPSHOTS,
+                        (Consumer<KeyedStateHandle>)
+                                keyedState ->
+                                        assertThat(
+                                                keyedState,
+                                                instanceOf(
+                                                        IncrementalRemoteKeyedStateHandle.class))));

Review comment:
       Is this assertion enough to make sure that native savepoints are working?




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

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5886c2da10de2ef3e31c94b4bb56e1aceb6deceb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5886c2da10de2ef3e31c94b4bb56e1aceb6deceb Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501) 
   * 08906ab9a71f021c125f27122419a89a07be8ee3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688",
       "triggerID" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ed5d5d894e019177d00af8193653c69c21ccc725 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679) 
   * e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 08906ab9a71f021c125f27122419a89a07be8ee3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518) 
   * ed5d5d894e019177d00af8193653c69c21ccc725 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679) 
   * e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30518",
       "triggerID" : "08906ab9a71f021c125f27122419a89a07be8ee3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30679",
       "triggerID" : "ed5d5d894e019177d00af8193653c69c21ccc725",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688",
       "triggerID" : "e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e8bcbfd5a48fd8d3ca48ef7803867569214e0dbc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30688) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501",
       "triggerID" : "5886c2da10de2ef3e31c94b4bb56e1aceb6deceb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 528758ad9508f9fec5630b747fe9f454fbc30171 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310) 
   * 5886c2da10de2ef3e31c94b4bb56e1aceb6deceb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30501) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot edited a comment on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 528758ad9508f9fec5630b747fe9f454fbc30171 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=30310) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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



[GitHub] [flink] flinkbot commented on pull request #18539: [FLINK-25745] Support RocksDB incremental native savepoints

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #18539:
URL: https://github.com/apache/flink/pull/18539#issuecomment-1023172522


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "528758ad9508f9fec5630b747fe9f454fbc30171",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 528758ad9508f9fec5630b747fe9f454fbc30171 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot 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: issues-unsubscribe@flink.apache.org

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