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 2019/12/20 08:25:50 UTC

[GitHub] [flink] zhuzhurk opened a new pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

zhuzhurk opened a new pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646
 
 
   ## What is the purpose of the change
    
   JM crashes may happen when an external job cancel request comes when the job is still allocating slots.
   The root cause is that, the version of the canceled vertices are not incremented in the case of external job cancel request, and the pending slot requests are also not canceled in this case, so that the returned slot can be used to fulfill an outdated deployment, which finally triggers the fatal error.
   
   ## Brief change log
   
     - *increment all vertex versions in #failJob(), #cancel(), #suspend() in SchedulerBase*
   
   ## Verifying this change
   
     - *Added unit tests for job failing/canceling/suspending cases*
   
   ## 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, Yarn/Mesos, 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#discussion_r360835028
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
 ##########
 @@ -550,6 +550,53 @@ public void testInputConstraintALLPerf() throws Exception {
 		assertThat(duration, lessThan(timeout));
 	}
 
+	@Test
+	public void failJobWillIncrementVertexVersions() {
+		final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+		final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph);
+		final ExecutionVertexID onlyExecutionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0);
+
+		// suppress restarts so any task failure will lead to job failure
+		testRestartBackoffTimeStrategy.setCanRestart(false);
+		final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+		final ArchivedExecutionVertex archivedExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices());
+		final ExecutionAttemptID attemptId = archivedExecutionVertex.getCurrentExecutionAttempt().getAttemptId();
+
+		scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED));
+
+		final ExecutionVertexVersioner executionVertexVersioner = scheduler.getExecutionVertexVersioner();
 
 Review comment:
   Not needed. We can directly access `executionVertexVersioner` as it is a field of this test suite.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#discussion_r360835968
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
 ##########
 @@ -550,6 +550,53 @@ public void testInputConstraintALLPerf() throws Exception {
 		assertThat(duration, lessThan(timeout));
 	}
 
+	@Test
+	public void failJobWillIncrementVertexVersions() {
+		final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+		final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph);
+		final ExecutionVertexID onlyExecutionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0);
+
+		// suppress restarts so any task failure will lead to job failure
+		testRestartBackoffTimeStrategy.setCanRestart(false);
+		final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+		final ArchivedExecutionVertex archivedExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices());
+		final ExecutionAttemptID attemptId = archivedExecutionVertex.getCurrentExecutionAttempt().getAttemptId();
+
+		scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED));
 
 Review comment:
   I think the test can be simplified by directly calling
   
   ```
   scheduler.failJob(new FlinkException("Test failure."));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#discussion_r360835260
 
 

 ##########
 File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
 ##########
 @@ -397,6 +407,18 @@ protected JobGraph getJobGraph() {
 
 	protected abstract long getNumberOfRestarts();
 
+	private Map<ExecutionVertexID, ExecutionVertexVersion> incrementVersionsOfAllVertices() {
+		return executionVertexVersioner.recordVertexModifications(
+			IterableUtils.toStream(schedulingTopology.getVertices())
+				.map(SchedulingExecutionVertex::getId)
+				.collect(Collectors.toSet()));
+	}
+
+	@VisibleForTesting
+	ExecutionVertexVersioner getExecutionVertexVersioner() {
+		return executionVertexVersioner;
+	}
 
 Review comment:
   Can be removed, as we pass in the `executionVertexVersioner` and can keep a reference to 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   * 2df76740f2fe2911b43e5929d3e9c8a0a5938989 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142099934 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   * 2df76740f2fe2911b43e5929d3e9c8a0a5938989 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142099934) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] tillrohrmann closed pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
tillrohrmann closed pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142099934 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   * 2df76740f2fe2911b43e5929d3e9c8a0a5938989 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142099934) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142099934 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   Hash:2df76740f2fe2911b43e5929d3e9c8a0a5938989 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860 TriggerType:PUSH TriggerID:2df76740f2fe2911b43e5929d3e9c8a0a5938989
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   * 2df76740f2fe2911b43e5929d3e9c8a0a5938989 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142099934) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3860) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#discussion_r360839936
 
 

 ##########
 File path: flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java
 ##########
 @@ -550,6 +550,53 @@ public void testInputConstraintALLPerf() throws Exception {
 		assertThat(duration, lessThan(timeout));
 	}
 
+	@Test
+	public void failJobWillIncrementVertexVersions() {
+		final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+		final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph);
+		final ExecutionVertexID onlyExecutionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0);
+
+		// suppress restarts so any task failure will lead to job failure
+		testRestartBackoffTimeStrategy.setCanRestart(false);
+		final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
+
+		final ArchivedExecutionVertex archivedExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices());
+		final ExecutionAttemptID attemptId = archivedExecutionVertex.getCurrentExecutionAttempt().getAttemptId();
+
+		scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED));
+
+		final ExecutionVertexVersioner executionVertexVersioner = scheduler.getExecutionVertexVersioner();
+		assertTrue(executionVertexVersioner.isModified(new ExecutionVertexVersion(onlyExecutionVertexId, 1)));
 
 Review comment:
   I think this encodes some implementation details that the first recorded version starts with `1`. It would be better to ask the `executionVertexVersioner` about the `ExecutionVertexVersion` before triggering the `failJob` call and then compare this value against the currently recorded value.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10646: [FLINK-15320][runtime] Increment versions of all vertices when failing/canceling/suspending a job
URL: https://github.com/apache/flink/pull/10646#issuecomment-567852821
 
 
   <!--
   Meta data
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/141886760 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   Hash:f4ea099d8da02d43b177212c0df6e993ee6004ea Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814 TriggerType:PUSH TriggerID:f4ea099d8da02d43b177212c0df6e993ee6004ea
   -->
   ## CI report:
   
   * f4ea099d8da02d43b177212c0df6e993ee6004ea Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/141886760) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=3814) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services