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/11 17:55:46 UTC

[GitHub] [flink] dmvk opened a new pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

dmvk opened a new pull request #18332:
URL: https://github.com/apache/flink/pull/18332


   https://issues.apache.org/jira/projects/FLINK/issues/FLINK-25427


-- 
This is an automated message from the Apache Git Service.
To 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] akalash commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       I agree that the contract looks strange. Why can not we indeed clone the JobGraph in submitJob?(it is better not to clone but use builder and so but as I understand it is not easy to do so now)

##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
##########
@@ -431,36 +430,48 @@ public void invoke(Long value) {
                                 .build());
         cluster.before();
         try {
-            final JobID jobID1 = new JobID();
-            jobGraph.setJobID(jobID1);
-            cluster.getClusterClient().submitJob(jobGraph).get();
-            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID1, false);
+            final JobID firstJobId = new JobID();
+            final JobGraph firstJobGraph = InstantiationUtil.clone(jobGraph);
+            firstJobGraph.setJobID(firstJobId);
+            cluster.getClusterClient().submitJob(firstJobGraph).get();
+            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), firstJobId, false);
             // wait for some records to be processed before taking the checkpoint
             counter.get().await();
-            final String firstCheckpoint = cluster.getMiniCluster().triggerCheckpoint(jobID1).get();
-
-            cluster.getClusterClient().cancel(jobID1).get();
-            jobGraph.setSavepointRestoreSettings(
+            final String firstCheckpoint =
+                    cluster.getMiniCluster().triggerCheckpoint(firstJobId).get();
+            cluster.getClusterClient().cancel(firstJobId).get();
+            CommonTestUtils.waitForJobStatus(

Review comment:
       I don't fully understand why we need it here. if we clone JobGraph then everything works perfectly fine even without this waiting. It seems for me that the awaiting on `cancel's future` is enough to be sure that the next submit with the new JobGraph will be successful. Did I miss some rare race condition?




-- 
This is an automated message from the Apache Git Service.
To 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] akalash commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
##########
@@ -431,36 +430,48 @@ public void invoke(Long value) {
                                 .build());
         cluster.before();
         try {
-            final JobID jobID1 = new JobID();
-            jobGraph.setJobID(jobID1);
-            cluster.getClusterClient().submitJob(jobGraph).get();
-            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID1, false);
+            final JobID firstJobId = new JobID();
+            final JobGraph firstJobGraph = InstantiationUtil.clone(jobGraph);

Review comment:
       As I understand, you don't need `clone` here anymore

##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
##########
@@ -431,36 +430,48 @@ public void invoke(Long value) {
                                 .build());
         cluster.before();
         try {
-            final JobID jobID1 = new JobID();
-            jobGraph.setJobID(jobID1);
-            cluster.getClusterClient().submitJob(jobGraph).get();
-            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID1, false);
+            final JobID firstJobId = new JobID();
+            final JobGraph firstJobGraph = InstantiationUtil.clone(jobGraph);
+            firstJobGraph.setJobID(firstJobId);
+            cluster.getClusterClient().submitJob(firstJobGraph).get();
+            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), firstJobId, false);
             // wait for some records to be processed before taking the checkpoint
             counter.get().await();
-            final String firstCheckpoint = cluster.getMiniCluster().triggerCheckpoint(jobID1).get();
-
-            cluster.getClusterClient().cancel(jobID1).get();
-            jobGraph.setSavepointRestoreSettings(
+            final String firstCheckpoint =
+                    cluster.getMiniCluster().triggerCheckpoint(firstJobId).get();
+            cluster.getClusterClient().cancel(firstJobId).get();
+            CommonTestUtils.waitForJobStatus(

Review comment:
       Since you cloned the object inside of submit. Do I understand correctly, that we can avoid `waitForJobStatus`? (I understand that it also makes sense, but I just don't want to overcomplicate the code if it is not necessary)




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29776",
       "triggerID" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b933ff1fe3f83687c565a178d7cc4bcc97ddd248 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712) 
   * 4d71c2fa1ebed8d839c4e9d08968660a4916fd1f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29776) 
   
   <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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a8429ae65d3c031d8552749618fc2e38f9c87acd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436) 
   
   <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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
##########
@@ -431,36 +430,48 @@ public void invoke(Long value) {
                                 .build());
         cluster.before();
         try {
-            final JobID jobID1 = new JobID();
-            jobGraph.setJobID(jobID1);
-            cluster.getClusterClient().submitJob(jobGraph).get();
-            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), jobID1, false);
+            final JobID firstJobId = new JobID();
+            final JobGraph firstJobGraph = InstantiationUtil.clone(jobGraph);
+            firstJobGraph.setJobID(firstJobId);
+            cluster.getClusterClient().submitJob(firstJobGraph).get();
+            CommonTestUtils.waitForAllTaskRunning(cluster.getMiniCluster(), firstJobId, false);
             // wait for some records to be processed before taking the checkpoint
             counter.get().await();
-            final String firstCheckpoint = cluster.getMiniCluster().triggerCheckpoint(jobID1).get();
-
-            cluster.getClusterClient().cancel(jobID1).get();
-            jobGraph.setSavepointRestoreSettings(
+            final String firstCheckpoint =
+                    cluster.getMiniCluster().triggerCheckpoint(firstJobId).get();
+            cluster.getClusterClient().cancel(firstJobId).get();
+            CommonTestUtils.waitForJobStatus(

Review comment:
       This would fix the test as well, even without cloning the `JobGraph`, because after this point the mutations wouldn't affect the dispatcher.




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 899e78f146881ebfc1a034522182a528599daebc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277) 
   * 5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300) 
   
   <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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a8429ae65d3c031d8552749618fc2e38f9c87acd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436) 
   * b933ff1fe3f83687c565a178d7cc4bcc97ddd248 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712) 
   
   <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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       The idea here, was to guard test authors for running into the same trap and identify other tests that may suffer from this.




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300) 
   * a8429ae65d3c031d8552749618fc2e38f9c87acd 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300) 
   * a8429ae65d3c031d8552749618fc2e38f9c87acd Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436) 
   
   <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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300) 
   
   <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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 899e78f146881ebfc1a034522182a528599daebc 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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       I have to double check, I think this might actually also be the case for application / per-job mode, but that might be OK as the user isn't really exposed to this API.
   
   Even if that's true, I'd be OK with cloning a JobGraph here, it has actually been my first approach.
   
   Overall the best solution would be simply making the `JobGraph` immutable, but that's a bigger story :/




-- 
This is an automated message from the Apache Git Service.
To 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] dmvk commented on pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   Needs another rebase due to conflict in MiniCluster


-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 899e78f146881ebfc1a034522182a528599daebc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277) 
   * 5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29776",
       "triggerID" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4d71c2fa1ebed8d839c4e9d08968660a4916fd1f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29776) 
   
   <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] dmvk commented on pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   @dawidwys yes please, I was waiting for CI to get green to ping you :)


-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 899e78f146881ebfc1a034522182a528599daebc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277) 
   
   <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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       Yes it should be as the job graph needs to send over the network so it's effectively cloned, so any mutations to the JobGraph on the client side wouldn't affect the instance used by Dispatcher.
   
   As I've already suggested, the other option would be simply cloning the job graph here.
   
   @zentol do you have any thoughts on this one?




-- 
This is an automated message from the Apache Git Service.
To 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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       The idea here, was to guard test authors from running into the same trap and identify other tests that may suffer from this.




-- 
This is an automated message from the Apache Git Service.
To 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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       Fixed




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d71c2fa1ebed8d839c4e9d08968660a4916fd1f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b933ff1fe3f83687c565a178d7cc4bcc97ddd248 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712) 
   * 4d71c2fa1ebed8d839c4e9d08968660a4916fd1f 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] dmvk commented on pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   @dawidwys Thanks for the review. I've addressed your comments, PTAL


-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   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 899e78f146881ebfc1a034522182a528599daebc (Tue Jan 11 17:59:14 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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       Yes it should be as the job graph needs to be send over the network so it's effectively cloned, any mutations to the JobGraph on the client side wouldn't affect the instance used by Dispatcher.
   
   As I've already suggested, the other option would be simply cloning the job graph here.
   
   @zentol do you have any thoughts on this one?




-- 
This is an automated message from the Apache Git Service.
To 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] dmvk commented on a change in pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java
##########
@@ -653,14 +653,7 @@ public void testOutOfMemoryErrorMessageEnrichmentInJobVertexInitialization() thr
         try (final MiniCluster miniCluster = new MiniCluster(cfg)) {
             miniCluster.start();
 
-            final JobVertex failingJobVertex =
-                    new JobVertex("FailingInFinalization") {
-
-                        @Override
-                        public void initializeOnMaster(ClassLoader loader) {
-                            throw new OutOfMemoryError("Java heap space");
-                        }
-                    };
+            final JobVertex failingJobVertex = new OutOfMemoryInInitializationVertex();

Review comment:
       This makes the JobGraph serializable




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       Is that really a contract we want to enforce? Why are you concerned with mutating the `JobGraph` between submissions? If I understand it correctly, it would be safe to do so with a proper cluster.




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java
##########
@@ -1175,6 +1185,31 @@ private void terminateMiniClusterServices() throws Exception {
         }
     }
 
+    /**
+     * Prevent multiple submission of the same JobGraph that has been mutated in between

Review comment:
       Yes, I agree with the analysis. However, I'd rather go with the serialization approach you suggested. Otherwise we are imposing a contract on the MiniCluster that is different from regular clusters. At least that's my take. WDYT?




-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 899e78f146881ebfc1a034522182a528599daebc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277) 
   
   <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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b933ff1fe3f83687c565a178d7cc4bcc97ddd248 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29712) 
   
   <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 merged pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   


-- 
This is an automated message from the Apache Git Service.
To 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 #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "899e78f146881ebfc1a034522182a528599daebc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29277",
       "triggerID" : "899e78f146881ebfc1a034522182a528599daebc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29300",
       "triggerID" : "5e2c5dd3e14dbe4e201612a1cbd2816328ea8f77",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436",
       "triggerID" : "a8429ae65d3c031d8552749618fc2e38f9c87acd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b933ff1fe3f83687c565a178d7cc4bcc97ddd248",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a8429ae65d3c031d8552749618fc2e38f9c87acd Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=29436) 
   * b933ff1fe3f83687c565a178d7cc4bcc97ddd248 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 pull request #18332: [FLINK-25427] Fix SavepointITCase.testTriggerSavepointAndResumeWithNo…

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


   Shall I merge it @dmvk ?


-- 
This is an automated message from the Apache Git Service.
To 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